diff --git a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java index 1fdef686f6ee..506023b22a7d 100644 --- a/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java +++ b/hudi-aws/src/main/java/org/apache/hudi/aws/sync/AWSGlueCatalogSyncClient.java @@ -894,7 +894,7 @@ public void close() { @Override public void updateLastCommitTimeSynced(String tableName) { HoodieTimeline activeTimeline = getActiveTimeline(); - Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::getTimestamp); + Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::requestedTime); Option lastCommitCompletionSynced = activeTimeline .getInstantsOrderedByCompletionTime() .skip(activeTimeline.countInstants() - 1) diff --git a/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java b/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java index 0ba4a0496f9e..0dc0890be498 100644 --- a/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java +++ b/hudi-aws/src/test/java/org/apache/hudi/aws/testutils/GlueTestUtil.java @@ -23,7 +23,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantFileNameGenerator; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.hive.HiveSyncConfig; import org.apache.hudi.hive.SlashEncodedDayPartitionValueExtractor; @@ -112,7 +112,7 @@ public static void createHoodieTable() throws IOException { String instantTime = "101"; HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(false); - createMetaFile(basePath, HoodieTimeline.makeCommitFileName(instantTime), commitMetadata); + createMetaFile(basePath, new DefaultInstantFileNameGenerator().makeCommitFileName(instantTime), commitMetadata); } public static MessageType getSimpleSchema() { diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java index 05be8049121e..ff504a7f3677 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/HoodieCLI.java @@ -23,8 +23,6 @@ import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; -import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StorageConfiguration; @@ -48,7 +46,6 @@ public class HoodieCLI { public static String basePath; protected static HoodieTableMetaClient tableMetadata; public static HoodieTableMetaClient syncTableMetadata; - public static TimelineLayoutVersion layoutVersion; public static TempViewProvider tempViewProvider; /** @@ -74,11 +71,6 @@ private static void setBasePath(String basePath) { HoodieCLI.basePath = basePath; } - private static void setLayoutVersion(Integer layoutVersion) { - HoodieCLI.layoutVersion = new TimelineLayoutVersion( - (layoutVersion == null) ? TimelineLayoutVersion.CURR_VERSION : layoutVersion); - } - public static boolean initConf() { if (HoodieCLI.conf == null) { HoodieCLI.conf = HadoopFSUtils.getStorageConf( @@ -101,12 +93,11 @@ public static void refreshTableMetadata() { .setConf(HoodieCLI.conf.newInstance()).setBasePath(basePath).setLoadActiveTimelineOnLoad(false) .setConsistencyGuardConfig(HoodieCLI.consistencyGuardConfig) .setTimeGeneratorConfig(timeGeneratorConfig == null ? HoodieTimeGeneratorConfig.defaultConfig(basePath) : timeGeneratorConfig) - .setLayoutVersion(Option.of(layoutVersion)).build()); + .build()); } - public static void connectTo(String basePath, Integer layoutVersion) { + public static void connectTo(String basePath) { setBasePath(basePath); - setLayoutVersion(layoutVersion); refreshTableMetadata(); } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java index 09d17d4a9297..d5a00437a22d 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/CleansCommand.java @@ -28,10 +28,10 @@ import org.apache.hudi.cli.utils.InputStreamConsumer; import org.apache.hudi.cli.utils.SparkUtil; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; + import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.utilities.UtilHelpers; @@ -72,14 +72,14 @@ public String showCleans( defaultValue = "false") final boolean headerOnly) throws IOException { - HoodieDefaultTimeline activeTimeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); + HoodieTimeline activeTimeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); List cleans = timeline.getReverseOrderedInstants().collect(Collectors.toList()); List rows = new ArrayList<>(); for (HoodieInstant clean : cleans) { HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); - rows.add(new Comparable[] {clean.getTimestamp(), cleanMetadata.getEarliestCommitToRetain(), + rows.add(new Comparable[] {clean.requestedTime(), cleanMetadata.getEarliestCommitToRetain(), cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()}); } @@ -103,7 +103,7 @@ public String showCleanPartitions( HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCleanerTimeline().filterCompletedInstants(); - HoodieInstant cleanInstant = new HoodieInstant(false, HoodieTimeline.CLEAN_ACTION, instantTime); + HoodieInstant cleanInstant = HoodieCLI.getTableMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLEAN_ACTION, instantTime); if (!timeline.containsInstant(cleanInstant)) { return "Clean " + instantTime + " not found in metadata " + timeline; 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 be7aceb039f8..3b42edc383a5 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 @@ -25,10 +25,10 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparator; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; @@ -48,6 +48,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.TimelineUtils.getTimeline; /** @@ -56,23 +58,25 @@ @ShellComponent public class CommitsCommand { - private String printCommits(HoodieDefaultTimeline timeline, + private String printCommits(HoodieTimeline timeline, final Integer limit, final String sortByField, final boolean descending, final boolean headerOnly, final String tempTableName) throws IOException { final List rows = new ArrayList<>(); + InstantComparator instantComparator = HoodieCLI.getTableMetaClient().getTimelineLayout().getInstantComparator(); final List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstantsAsStream().sorted(HoodieInstant.INSTANT_TIME_COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { if (timeline.getInstantDetails(commit).isPresent()) { - final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + final HoodieCommitMetadata commitMetadata = HoodieCLI.getTableMetaClient().getCommitMetadataSerDe().deserialize( + commit, timeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); - rows.add(new Comparable[] {commit.getTimestamp(), + rows.add(new Comparable[] {commit.requestedTime(), commitMetadata.fetchTotalBytesWritten(), commitMetadata.fetchTotalFilesInsert(), commitMetadata.fetchTotalFilesUpdated(), @@ -94,20 +98,22 @@ private String printCommits(HoodieDefaultTimeline timeline, limit, headerOnly, rows, tempTableName); } - private String printCommitsWithMetadata(HoodieDefaultTimeline timeline, + private String printCommitsWithMetadata(HoodieTimeline timeline, final Integer limit, final String sortByField, final boolean descending, final boolean headerOnly, final String tempTableName, final String partition) throws IOException { final List rows = new ArrayList<>(); + InstantComparator instantComparator = HoodieCLI.getTableMetaClient().getTimelineLayout().getInstantComparator(); final List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstantsAsStream().sorted(HoodieInstant.INSTANT_TIME_COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { if (timeline.getInstantDetails(commit).isPresent()) { - final HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + final HoodieCommitMetadata commitMetadata = HoodieCLI.getTableMetaClient().getCommitMetadataSerDe().deserialize( + commit, timeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); @@ -115,7 +121,7 @@ private String printCommitsWithMetadata(HoodieDefaultTimeline timeline, commitMetadata.getPartitionToWriteStats().entrySet()) { for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { if (StringUtils.isNullOrEmpty(partition) || partition.equals(hoodieWriteStat.getPartitionPath())) { - rows.add(new Comparable[] {commit.getAction(), commit.getTimestamp(), hoodieWriteStat.getPartitionPath(), + rows.add(new Comparable[] {commit.getAction(), commit.requestedTime(), hoodieWriteStat.getPartitionPath(), hoodieWriteStat.getFileId(), hoodieWriteStat.getPrevCommit(), hoodieWriteStat.getNumWrites(), hoodieWriteStat.getNumInserts(), hoodieWriteStat.getNumDeletes(), hoodieWriteStat.getNumUpdateWrites(), hoodieWriteStat.getTotalWriteErrors(), @@ -155,7 +161,7 @@ public String showCommits( defaultValue = "false") final boolean includeArchivedTimeline) throws IOException { - HoodieDefaultTimeline timeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); + HoodieTimeline timeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); if (includeExtraMetadata) { return printCommitsWithMetadata(timeline, limit, sortByField, descending, headerOnly, exportTableName, partition); } else { @@ -188,7 +194,7 @@ public String showArchivedCommits( HoodieArchivedTimeline archivedTimeline = HoodieCLI.getTableMetaClient().getArchivedTimeline(); try { archivedTimeline.loadInstantDetailsInMemory(startTs, endTs); - HoodieDefaultTimeline timelineRange = archivedTimeline.findInstantsInRange(startTs, endTs); + HoodieTimeline timelineRange = (HoodieTimeline)archivedTimeline.findInstantsInRange(startTs, endTs); if (includeExtraMetadata) { return printCommitsWithMetadata(timelineRange, limit, sortByField, descending, headerOnly, exportTableName, partition); } else { @@ -214,7 +220,7 @@ public String showCommitPartitions( defaultValue = "false") final boolean includeArchivedTimeline) throws Exception { - HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); + HoodieTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants(); Option hoodieInstantOption = getCommitForInstant(timeline, instantTime); @@ -282,7 +288,7 @@ public String showWriteStats( defaultValue = "false") final boolean includeArchivedTimeline) throws Exception { - HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); + HoodieTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants(); Option hoodieInstantOption = getCommitForInstant(timeline, instantTime); @@ -328,7 +334,7 @@ public String showCommitFiles( defaultValue = "false") final boolean includeArchivedTimeline) throws Exception { - HoodieDefaultTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); + HoodieTimeline defaultTimeline = getTimeline(HoodieCLI.getTableMetaClient(), includeArchivedTimeline); HoodieTimeline timeline = defaultTimeline.getCommitsTimeline().filterCompletedInstants(); Option hoodieInstantOption = getCommitForInstant(timeline, instantTime); @@ -373,20 +379,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.getInstantsAsStream().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0"; + targetTimeline.getInstantsAsStream().iterator().hasNext() ? targetTimeline.lastInstant().get().requestedTime() : "0"; String sourceLatestCommit = - sourceTimeline.getInstantsAsStream().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0"; + sourceTimeline.getInstantsAsStream().iterator().hasNext() ? sourceTimeline.lastInstant().get().requestedTime() : "0"; if (sourceLatestCommit != null - && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) { + && compareTimestamps(targetLatestCommit, GREATER_THAN, sourceLatestCommit)) { // source is behind the target List commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE) - .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::requestedTime).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) - .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toList()); return "Source " + source.getTableConfig().getTableName() + " is ahead by " + commitsToCatchup.size() + " commits. Commits to catch up - " + commitsToCatchup; } @@ -406,9 +412,9 @@ public String syncCommits(@ShellOption(value = {"--path"}, help = "Path of the t * */ private Option getCommitForInstant(HoodieTimeline timeline, String instantTime) { List instants = Arrays.asList( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)); + HoodieCLI.getTableMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime), + HoodieCLI.getTableMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), + HoodieCLI.getTableMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)); return Option.fromJavaOptional(instants.stream().filter(timeline::containsInstant).findAny()); } 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 c0993c7b1ed5..f98ea7cecbb8 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 @@ -31,11 +31,11 @@ import org.apache.hudi.client.CompactionAdminClient.ValidationOpResult; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -124,9 +124,10 @@ public String compactionShow( throws Exception { HoodieTableMetaClient client = checkAndGetMetaClient(); HoodieActiveTimeline activeTimeline = client.getActiveTimeline(); + InstantGenerator instantGenerator = client.getInstantGenerator(); HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan( activeTimeline.readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get()); + instantGenerator.getCompactionRequestedInstant(compactionInstantTime)).get()); return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition); } @@ -175,7 +176,7 @@ public String compactionShowArchived( throws Exception { HoodieTableMetaClient client = checkAndGetMetaClient(); HoodieArchivedTimeline archivedTimeline = client.getArchivedTimeline(); - HoodieInstant instant = new HoodieInstant(HoodieInstant.State.COMPLETED, + HoodieInstant instant = client.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime); try { archivedTimeline.loadCompactionDetailsInMemory(compactionInstantTime); @@ -246,7 +247,7 @@ public String compact( Option firstPendingInstant = client.reloadActiveTimeline().filterCompletedAndCompactionInstants() .filter(instant -> instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)).firstInstant() - .map(HoodieInstant::getTimestamp); + .map(HoodieInstant::requestedTime); if (!firstPendingInstant.isPresent()) { return "NO PENDING COMPACTION TO RUN"; } @@ -306,7 +307,7 @@ public String compact( /** * Prints all compaction details. */ - private static String printAllCompactions(HoodieDefaultTimeline timeline, + private static String printAllCompactions(HoodieTimeline timeline, Function compactionPlanReader, boolean includeExtraMetadata, String sortByField, @@ -321,25 +322,25 @@ private static String printAllCompactions(HoodieDefaultTimeline timeline, .collect(Collectors.toList()); Set committedInstants = timeline.getCommitAndReplaceTimeline().filterCompletedInstants() - .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + .getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); List rows = new ArrayList<>(); for (Pair compactionPlan : compactionPlans) { HoodieCompactionPlan plan = compactionPlan.getRight(); HoodieInstant instant = compactionPlan.getLeft(); final HoodieInstant.State state; - if (committedInstants.contains(instant.getTimestamp())) { + if (committedInstants.contains(instant.requestedTime())) { state = HoodieInstant.State.COMPLETED; } else { state = instant.getState(); } if (includeExtraMetadata) { - rows.add(new Comparable[] {instant.getTimestamp(), state.toString(), + rows.add(new Comparable[] {instant.requestedTime(), state.toString(), plan.getOperations() == null ? 0 : plan.getOperations().size(), plan.getExtraMetadata().toString()}); } else { - rows.add(new Comparable[] {instant.getTimestamp(), state.toString(), + rows.add(new Comparable[] {instant.requestedTime(), state.toString(), plan.getOperations() == null ? 0 : plan.getOperations().size()}); } } @@ -360,7 +361,7 @@ private static String printAllCompactions(HoodieDefaultTimeline timeline, * We can make these read methods part of HoodieDefaultTimeline and override where necessary. But the * BiFunction below has 'hacky' exception blocks, so restricting it to CLI. */ - private + private Function compactionPlanReader( BiFunction f, T timeline) { @@ -381,20 +382,21 @@ private HoodieCompactionPlan readCompactionPlanForArchivedTimeline(HoodieArchive */ private HoodieCompactionPlan readCompactionPlanForActiveTimeline(HoodieActiveTimeline activeTimeline, HoodieInstant instant) { + InstantGenerator instantGenerator = HoodieCLI.getTableMetaClient().getInstantGenerator(); try { if (!HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction())) { try { // This could be a completed compaction. Assume a compaction request file is present but skip if fails return TimelineMetadataUtils.deserializeCompactionPlan( activeTimeline.readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); + instantGenerator.getCompactionRequestedInstant(instant.requestedTime())).get()); } catch (HoodieIOException ioe) { // SKIP return null; } } else { return TimelineMetadataUtils.deserializeCompactionPlan(activeTimeline.readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(instant.getTimestamp())).get()); + instantGenerator.getCompactionRequestedInstant(instant.requestedTime())).get()); } } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); 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 4ae843ef9b9b..617a820f87fc 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 @@ -19,13 +19,16 @@ package org.apache.hudi.cli.commands; +import org.apache.hudi.cli.HoodieCLI; import org.apache.hudi.cli.HoodiePrintHelper; import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.utils.CLIUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantComparator; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; @@ -65,7 +68,7 @@ public String diffFile( @ShellOption(value = {"--headeronly"}, help = "Print Header Only", defaultValue = "false") final boolean headerOnly, @ShellOption(value = {"--includeArchivedTimeline"}, help = "Include archived commits as well", defaultValue = "false") final boolean includeArchivedTimeline) throws IOException { - HoodieDefaultTimeline timeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); + HoodieTimeline timeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); return printCommitsWithMetadataForFileId(timeline, limit, sortByField, descending, headerOnly, "", fileId); } @@ -82,11 +85,11 @@ public String diffPartition( @ShellOption(value = {"--headeronly"}, help = "Print Header Only", defaultValue = "false") final boolean headerOnly, @ShellOption(value = {"--includeArchivedTimeline"}, help = "Include archived commits as well", defaultValue = "false") final boolean includeArchivedTimeline) throws IOException { - HoodieDefaultTimeline timeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); + HoodieTimeline timeline = CLIUtils.getTimelineInRange(startTs, endTs, includeArchivedTimeline); return printCommitsWithMetadataForPartition(timeline, limit, sortByField, descending, headerOnly, "", partitionPath); } - private String printCommitsWithMetadataForFileId(HoodieDefaultTimeline timeline, + private String printCommitsWithMetadataForFileId(HoodieTimeline timeline, final Integer limit, final String sortByField, final boolean descending, @@ -96,7 +99,7 @@ private String printCommitsWithMetadataForFileId(HoodieDefaultTimeline timeline, return printDiffWithMetadata(timeline, limit, sortByField, descending, headerOnly, tempTableName, fileId, FILE_ID_CHECKER); } - private String printCommitsWithMetadataForPartition(HoodieDefaultTimeline timeline, + private String printCommitsWithMetadataForPartition(HoodieTimeline timeline, final Integer limit, final String sortByField, final boolean descending, @@ -106,16 +109,18 @@ private String printCommitsWithMetadataForPartition(HoodieDefaultTimeline timeli return printDiffWithMetadata(timeline, limit, sortByField, descending, headerOnly, tempTableName, partition, PARTITION_CHECKER); } - private String printDiffWithMetadata(HoodieDefaultTimeline timeline, Integer limit, String sortByField, boolean descending, boolean headerOnly, String tempTableName, String diffEntity, + private String printDiffWithMetadata(HoodieTimeline timeline, Integer limit, String sortByField, boolean descending, boolean headerOnly, String tempTableName, String diffEntity, BiFunction diffEntityChecker) throws IOException { + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); List rows = new ArrayList<>(); + InstantComparator instantComparator = HoodieCLI.getTableMetaClient().getTimelineLayout().getInstantComparator(); List commits = timeline.getCommitsTimeline().filterCompletedInstants() - .getInstantsAsStream().sorted(HoodieInstant.INSTANT_TIME_COMPARATOR.reversed()).collect(Collectors.toList()); + .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { Option instantDetails = timeline.getInstantDetails(commit); if (instantDetails.isPresent()) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(instantDetails.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize(commit, instantDetails.get(), HoodieCommitMetadata.class); for (Map.Entry> partitionWriteStat : commitMetadata.getPartitionToWriteStats().entrySet()) { for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { @@ -139,7 +144,7 @@ private void populateRows(List rows, HoodieInstant commit, HoodieW if (checker.apply(hoodieWriteStat, value)) { rows.add(new Comparable[] { commit.getAction(), - commit.getTimestamp(), + commit.requestedTime(), hoodieWriteStat.getPartitionPath(), hoodieWriteStat.getFileId(), hoodieWriteStat.getPrevCommit(), 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 b4d33692b02c..c17114cd1154 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 @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.exception.HoodieException; @@ -194,9 +195,10 @@ private int copyNonArchivedInstants(List instants, int limit, Str } final HoodieTableMetaClient metaClient = HoodieCLI.getTableMetaClient(); + final InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); final HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); for (HoodieInstant instant : instants) { - String localPath = localFolder + StoragePath.SEPARATOR + instant.getFileName(); + String localPath = localFolder + StoragePath.SEPARATOR + instantFileNameGenerator.getFileName(instant); byte[] data = null; switch (instant.getAction()) { 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 a6a3048615bd..6a03155568e7 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 @@ -26,9 +26,9 @@ import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; @@ -51,6 +51,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; + /** * CLI command to display file system options. */ @@ -150,7 +153,7 @@ public String showLatestFileSlices( } else { if (maxInstant.isEmpty()) { maxInstant = HoodieCLI.getTableMetaClient().getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant() - .get().getTimestamp(); + .get().requestedTime(); } fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn(partition, maxInstant); } @@ -263,14 +266,14 @@ private HoodieTableFileSystemView buildFileSystemView(String globRegex, String m if (!maxInstant.isEmpty()) { final BiPredicate predicate; if (includeMaxInstant) { - predicate = HoodieTimeline.GREATER_THAN_OR_EQUALS; + predicate = GREATER_THAN_OR_EQUALS; } else { - predicate = HoodieTimeline.GREATER_THAN; + predicate = GREATER_THAN; } - instantsStream = instantsStream.filter(is -> predicate.test(maxInstant, is.getTimestamp())); + instantsStream = instantsStream.filter(is -> predicate.test(maxInstant, is.requestedTime())); } - - HoodieTimeline filteredTimeline = new HoodieDefaultTimeline(instantsStream, + TimelineFactory timelineFactory = metaClient.getTimelineLayout().getTimelineFactory(); + HoodieTimeline filteredTimeline = timelineFactory.createDefaultTimeline(instantsStream, (Function> & Serializable) metaClient.getActiveTimeline()::getInstantDetails); return new HoodieTableFileSystemView(metaClient, filteredTimeline, pathInfoList); } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java index 4d1979a82097..a33e03472bce 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/HoodieLogFileCommand.java @@ -233,7 +233,7 @@ storage, new StoragePath(logFilePathPattern)).stream() .withReaderSchema(readerSchema) .withLatestInstantTime( client.getActiveTimeline() - .getCommitAndReplaceTimeline().lastInstant().get().getTimestamp()) + .getCommitAndReplaceTimeline().lastInstant().get().requestedTime()) .withReverseReader( Boolean.parseBoolean( HoodieReaderConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue())) 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 4e3219d18dbb..807249a43e99 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 @@ -34,6 +34,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.cli.utils.CommitUtil.countNewRecords; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * CLI command to display sync options. @@ -74,12 +76,12 @@ public String validateSync( } String targetLatestCommit = - targetTimeline.getInstants().iterator().hasNext() ? targetTimeline.lastInstant().get().getTimestamp() : "0"; + targetTimeline.getInstants().iterator().hasNext() ? targetTimeline.lastInstant().get().requestedTime() : "0"; String sourceLatestCommit = - sourceTimeline.getInstants().iterator().hasNext() ? sourceTimeline.lastInstant().get().getTimestamp() : "0"; + sourceTimeline.getInstants().iterator().hasNext() ? sourceTimeline.lastInstant().get().requestedTime() : "0"; if (sourceLatestCommit != null - && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) { + && compareTimestamps(targetLatestCommit, GREATER_THAN, sourceLatestCommit)) { // source is behind the target return getString(target, targetTimeline, source, sourceCount, targetCount, sourceLatestCommit); } else { @@ -97,7 +99,7 @@ private String getString(HoodieTableMetaClient target, HoodieTimeline targetTime + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount); } else { long newInserts = countNewRecords(target, - commitsToCatchup.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList())); + commitsToCatchup.stream().map(HoodieInstant::requestedTime).collect(Collectors.toList())); return "Count difference now is (count(" + target.getTableConfig().getTableName() + ") - count(" + source.getTableConfig().getTableName() + ") == " + (targetCount - sourceCount) + ". Catch up count is " + newInserts; 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 7d09ecc8dc95..f916ab62937f 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 @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.PartitionPathEncodeUtils; @@ -117,7 +118,7 @@ public String addPartitionMeta( HoodieTableMetaClient client = HoodieCLI.getTableMetaClient(); String latestCommit = - client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().getTimestamp(); + client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().requestedTime(); List partitionPaths = FSUtils.getAllPartitionFoldersThreeLevelsDown(HoodieCLI.storage, HoodieCLI.basePath); StoragePath basePath = client.getBasePath(); @@ -199,13 +200,13 @@ public void removeCorruptedPendingCleanAction() { CleanerUtils.getCleanerPlan(client, instant); } catch (AvroRuntimeException e) { LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant); - HoodieActiveTimeline.deleteInstantFile(client.getStorage(), client.getMetaPath(), - instant); + TimelineUtils.deleteInstantFile(client.getStorage(), client.getMetaPath(), + instant, client.getInstantFileNameGenerator()); } catch (IOException ioe) { if (ioe.getMessage().contains("Not an Avro data file")) { LOG.warn("Corruption found. Trying to remove corrupted clean instant file: " + instant); - HoodieActiveTimeline.deleteInstantFile(client.getStorage(), client.getMetaPath(), - instant); + TimelineUtils.deleteInstantFile(client.getStorage(), client.getMetaPath(), + instant, client.getInstantFileNameGenerator()); } else { throw new HoodieIOException(ioe.getMessage(), ioe); } @@ -243,7 +244,7 @@ public String migratePartitionMeta( Option baseFormatFile = HoodiePartitionMetadata.baseFormatMetaPathIfExists(HoodieCLI.storage, partition); String latestCommit = - client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().getTimestamp(); + client.getActiveTimeline().getCommitAndReplaceTimeline().lastInstant().get().requestedTime(); String[] row = new String[] { partitionPath, diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RestoresCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RestoresCommand.java index 550166ac0fa1..6f4cc3be910c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RestoresCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RestoresCommand.java @@ -80,10 +80,10 @@ public String showRestore( HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); List matchingInstants = activeTimeline.filterCompletedInstants().filter(completed -> - completed.getTimestamp().equals(restoreInstant)).getInstants(); + completed.requestedTime().equals(restoreInstant)).getInstants(); if (matchingInstants.isEmpty()) { matchingInstants = activeTimeline.filterInflights().filter(inflight -> - inflight.getTimestamp().equals(restoreInstant)).getInstants(); + inflight.requestedTime().equals(restoreInstant)).getInstants(); } // Assuming a single exact match is found in either completed or inflight instants @@ -113,15 +113,15 @@ private void addDetailsOfInflightRestore(HoodieActiveTimeline activeTimeline, Li HoodieInstant restoreInstant) throws IOException { HoodieRestorePlan restorePlan = getRestorePlan(activeTimeline, restoreInstant); for (HoodieInstantInfo instantToRollback : restorePlan.getInstantsToRollback()) { - Comparable[] dataRow = createDataRow(restoreInstant.getTimestamp(), instantToRollback.getCommitTime(), "", + Comparable[] dataRow = createDataRow(restoreInstant.requestedTime(), instantToRollback.getCommitTime(), "", restoreInstant.getState()); rows.add(dataRow); } } private HoodieRestorePlan getRestorePlan(HoodieActiveTimeline activeTimeline, HoodieInstant restoreInstant) throws IOException { - HoodieInstant instantKey = new HoodieInstant(HoodieInstant.State.REQUESTED, RESTORE_ACTION, - restoreInstant.getTimestamp()); + HoodieInstant instantKey = HoodieCLI.getTableMetaClient().createNewInstant(HoodieInstant.State.REQUESTED, RESTORE_ACTION, + restoreInstant.requestedTime()); Option instantDetails = activeTimeline.getInstantDetails(instantKey); HoodieRestorePlan restorePlan = TimelineMetadataUtils .deserializeAvroMetadata(instantDetails.get(), HoodieRestorePlan.class); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java index 1974ac2fd985..03c40ea58f5c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/RollbacksCommand.java @@ -26,7 +26,6 @@ import org.apache.hudi.cli.utils.InputStreamConsumer; import org.apache.hudi.cli.utils.SparkUtil; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -99,7 +98,7 @@ public String showRollback( HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); final List rows = new ArrayList<>(); HoodieRollbackMetadata metadata = TimelineMetadataUtils.deserializeAvroMetadata( - activeTimeline.getInstantDetails(new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)).get(), + activeTimeline.getInstantDetails(HoodieCLI.getTableMetaClient().createNewInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)).get(), HoodieRollbackMetadata.class); metadata.getPartitionMetadata().forEach((key, value) -> Stream .concat(value.getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)), @@ -134,7 +133,7 @@ public String rollbackCommit( help = "Enabling marker based rollback") final String rollbackUsingMarkers) throws Exception { HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); - HoodieTimeline filteredTimeline = activeTimeline.filter(instant -> instant.getTimestamp().equals(instantTime)); + HoodieTimeline filteredTimeline = activeTimeline.filter(instant -> instant.requestedTime().equals(instantTime)); if (filteredTimeline.empty()) { return "Commit " + instantTime + " not found in Commits " + activeTimeline; } 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 3cf185cde934..8b5ed2fd1a08 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 @@ -51,7 +51,7 @@ public String showSavepoints() { String[][] rows = new String[commits.size()][]; for (int i = 0; i < commits.size(); i++) { HoodieInstant commit = commits.get(i); - rows[i] = new String[] {commit.getTimestamp()}; + rows[i] = new String[] {commit.requestedTime()}; } return HoodiePrintHelper.print(new String[] {HoodieTableHeaderFields.HEADER_SAVEPOINT_TIME}, rows); } @@ -107,7 +107,7 @@ public String rollbackToSavepoint( } HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieTimeline timeline = activeTimeline.getCommitsTimeline().filterCompletedInstants(); - List instants = timeline.getInstantsAsStream().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList()); + List instants = timeline.getInstantsAsStream().filter(instant -> instant.requestedTime().equals(instantTime)).collect(Collectors.toList()); if (instants.isEmpty()) { return String.format("Commit %s not found in Commits %s", instantTime, timeline); @@ -141,7 +141,8 @@ public String deleteSavepoint( if (completedInstants.empty()) { throw new HoodieException("There are no completed savepoint to run delete"); } - HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime); + HoodieInstant savePoint = metaClient.createNewInstant(HoodieInstant.State.COMPLETED, + HoodieTimeline.SAVEPOINT_ACTION, instantTime); if (!completedInstants.containsInstant(savePoint)) { return String.format("Commit %s not found in Commits %s", instantTime, completedInstants); 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 ac6c9536a0dc..4b322de5caf9 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; @@ -73,14 +74,15 @@ public String writeAmplificationStats( List rows = new ArrayList<>(); DecimalFormat df = new DecimalFormat("#.00"); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); for (HoodieInstant instantTime : timeline.getInstants()) { String waf = "0"; - HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(instantTime).get(), + HoodieCommitMetadata commit = layout.getCommitMetadataSerDe().deserialize(instantTime, activeTimeline.getInstantDetails(instantTime).get(), HoodieCommitMetadata.class); if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); } - rows.add(new Comparable[] {instantTime.getTimestamp(), commit.fetchTotalUpdateRecordsWritten(), + rows.add(new Comparable[] {instantTime.requestedTime(), commit.fetchTotalUpdateRecordsWritten(), commit.fetchTotalRecordsWritten(), waf}); totalRecordsUpserted += commit.fetchTotalUpdateRecordsWritten(); totalRecordsWritten += commit.fetchTotalRecordsWritten(); diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java index 211d40132531..5309a9ffe313 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/TableCommand.java @@ -79,7 +79,6 @@ public class TableCommand { @ShellMethod(key = "connect", value = "Connect to a hoodie table") public String connect( @ShellOption(value = {"--path"}, help = "Base Path of the table") final String path, - @ShellOption(value = {"--layoutVersion"}, help = "Timeline Layout version", defaultValue = ShellOption.NULL) Integer layoutVersion, @ShellOption(value = {"--eventuallyConsistent"}, defaultValue = "false", help = "Enable eventual consistency") final boolean eventuallyConsistent, @ShellOption(value = {"--initialCheckIntervalMs"}, defaultValue = "2000", @@ -107,7 +106,7 @@ public String connect( .withDefaultLockProvider(useDefaultLockProvider) .build()); HoodieCLI.initConf(); - HoodieCLI.connectTo(path, layoutVersion); + HoodieCLI.connectTo(path); HoodieCLI.initFS(true); HoodieCLI.state = HoodieCLI.CLIState.TABLE; return "Metadata for table " + HoodieCLI.getTableMetaClient().getTableConfig().getTableName() + " loaded"; @@ -118,9 +117,8 @@ public String createTable( final String name, final String tableTypeStr, String archiveFolder, - Integer layoutVersion, final String payloadClass) throws IOException { - return createTable(path, name, tableTypeStr, archiveFolder, layoutVersion, HoodieTableVersion.current().versionCode(), payloadClass); + return createTable(path, name, tableTypeStr, archiveFolder, HoodieTableVersion.current().versionCode(), payloadClass); } /** @@ -139,8 +137,6 @@ public String createTable( help = "Hoodie Table Type. Must be one of : COPY_ON_WRITE or MERGE_ON_READ") final String tableTypeStr, @ShellOption(value = {"--archiveLogFolder"}, help = "Folder Name for storing archived timeline", defaultValue = ShellOption.NULL) String archiveFolder, - @ShellOption(value = {"--layoutVersion"}, help = "Specific Layout Version to use", - defaultValue = ShellOption.NULL) Integer layoutVersion, @ShellOption(value = {"--tableVersion"}, help = "Specific table Version to create table as", defaultValue = ShellOption.NULL) Integer tableVersion, @ShellOption(value = {"--payloadClass"}, defaultValue = "org.apache.hudi.common.model.HoodieAvroPayload", @@ -167,11 +163,10 @@ public String createTable( .setTableName(name) .setArchiveLogFolder(archiveFolder) .setPayloadClassName(payloadClass) - .setTimelineLayoutVersion(layoutVersion) .setTableVersion(tableVersion == null ? HoodieTableVersion.current().versionCode() : tableVersion) .initTable(HoodieCLI.conf.newInstance(), path); // Now connect to ensure loading works - return connect(path, layoutVersion, false, 0, 0, 0, + return connect(path, false, 0, 0, 0, "WAIT_TO_ADJUST_SKEW", 200L, true); } @@ -326,7 +321,7 @@ public String changeTableType( for (int i = 0; i < pendingCompactionInstants.size(); i++) { HoodieInstant compactionInstant = pendingCompactionInstants.get(i); LOG.info("compact {} instant {}", i + 1, compactionInstant); - String result = new CompactionCommand().compact(parallelism, "", master, sparkMemory, retry, compactionInstant.getTimestamp(), propsFilePath, configs); + String result = new CompactionCommand().compact(parallelism, "", master, sparkMemory, retry, compactionInstant.requestedTime(), propsFilePath, configs); LOG.info("compact instant {} result: {}", compactionInstant, result); if (!result.startsWith(CompactionCommand.COMPACTION_EXE_SUCCESSFUL)) { throw new HoodieException(String.format("Compact %s failed", compactionInstant)); 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 1f4089bf1a83..0b6cabdcdb99 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 @@ -26,9 +26,11 @@ import org.apache.hudi.cli.HoodieTableHeaderFields; import org.apache.hudi.cli.TableHeader; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparator; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.metadata.HoodieTableMetadata; @@ -84,13 +86,13 @@ public String showActive( HoodieTableMetaClient mtMetaClient = getMetadataTableMetaClient(metaClient); return printTimelineInfoWithMetadataTable( metaClient.getActiveTimeline(), mtMetaClient.getActiveTimeline(), - getInstantInfoFromTimeline(metaClient.getStorage(), metaClient.getMetaPath()), - getInstantInfoFromTimeline(mtMetaClient.getStorage(), mtMetaClient.getMetaPath()), + getInstantInfoFromTimeline(metaClient, metaClient.getStorage(), metaClient.getMetaPath()), + getInstantInfoFromTimeline(mtMetaClient, mtMetaClient.getStorage(), mtMetaClient.getMetaPath()), limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo); } return printTimelineInfo( metaClient.getActiveTimeline(), - getInstantInfoFromTimeline(metaClient.getStorage(), metaClient.getMetaPath()), + getInstantInfoFromTimeline(metaClient, metaClient.getStorage(), metaClient.getMetaPath()), limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo); } catch (IOException e) { e.printStackTrace(); @@ -113,7 +115,7 @@ public String showIncomplete( try { return printTimelineInfo( metaClient.getActiveTimeline().filterInflightsAndRequested(), - getInstantInfoFromTimeline(metaClient.getStorage(), metaClient.getMetaPath()), + getInstantInfoFromTimeline(metaClient, metaClient.getStorage(), metaClient.getMetaPath()), limit, sortByField, descending, headerOnly, true, showTimeSeconds, showRollbackInfo); } catch (IOException e) { e.printStackTrace(); @@ -135,7 +137,7 @@ public String metadataShowActive( try { return printTimelineInfo( metaClient.getActiveTimeline(), - getInstantInfoFromTimeline(metaClient.getStorage(), metaClient.getMetaPath()), + getInstantInfoFromTimeline(metaClient, metaClient.getStorage(), metaClient.getMetaPath()), limit, sortByField, descending, headerOnly, true, showTimeSeconds, false); } catch (IOException e) { e.printStackTrace(); @@ -157,7 +159,7 @@ public String metadataShowIncomplete( try { return printTimelineInfo( metaClient.getActiveTimeline().filterInflightsAndRequested(), - getInstantInfoFromTimeline(metaClient.getStorage(), metaClient.getMetaPath()), + getInstantInfoFromTimeline(metaClient, metaClient.getStorage(), metaClient.getMetaPath()), limit, sortByField, descending, headerOnly, true, showTimeSeconds, false); } catch (IOException e) { e.printStackTrace(); @@ -174,16 +176,21 @@ private HoodieTableMetaClient getMetadataTableMetaClient(HoodieTableMetaClient m } private Map> getInstantInfoFromTimeline( - HoodieStorage storage, StoragePath metaPath) throws IOException { + HoodieTableMetaClient metaClient, HoodieStorage storage, StoragePath metaPath) throws IOException { Map> instantMap = new HashMap<>(); + + final InstantFileNameParser instantFileNameParser = metaClient.getInstantFileNameParser(); + final InstantComparator instantComparator = metaClient.getTimelineLayout().getInstantComparator(); + final InstantGenerator instantGenerator = metaClient.getInstantGenerator(); + Stream instantStream = HoodieTableMetaClient.scanFiles(storage, metaPath, path -> { // Include only the meta files with extensions that needs to be included - String extension = HoodieInstant.getTimelineFileExtension(path.getName()); - return HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE.contains(extension); - }).stream().map(HoodieInstantWithModTime::new); + String extension = instantFileNameParser.getTimelineFileExtension(path.getName()); + return metaClient.getActiveTimeline().getValidExtensionsInActiveTimeline().contains(extension); + }).stream().map(storagePathInfo -> new HoodieInstantWithModTime(storagePathInfo, instantGenerator, instantComparator)); instantStream.forEach(instant -> { - instantMap.computeIfAbsent(instant.getTimestamp(), t -> new HashMap<>()) + instantMap.computeIfAbsent(instant.requestedTime(), t -> new HashMap<>()) .put(instant.getState(), instant); }); return instantMap; @@ -210,7 +217,7 @@ private String printTimelineInfo( Map> rollbackInfoMap = getRolledBackInstantInfo(timeline); final List rows = timeline.getInstantsAsStream().map(instant -> { Comparable[] row = new Comparable[6]; - String instantTimestamp = instant.getTimestamp(); + String instantTimestamp = instant.requestedTime(); String rollbackInfoString = showRollbackInfo ? getRollbackInfoString(Option.of(instant), timeline, rollbackInfoMap) : ""; @@ -294,14 +301,14 @@ private String printTimelineInfoWithMetadataTable( } private Option getInstant(HoodieTimeline timeline, String instantTimestamp) { - return timeline.filter(instant -> instant.getTimestamp().equals(instantTimestamp)).firstInstant(); + return timeline.filter(instant -> instant.requestedTime().equals(instantTimestamp)).firstInstant(); } private String getInstantToRollback(HoodieTimeline timeline, HoodieInstant instant) { try { if (instant.isInflight()) { - HoodieInstant instantToUse = new HoodieInstant( - HoodieInstant.State.REQUESTED, instant.getAction(), instant.getTimestamp()); + HoodieInstant instantToUse = HoodieCLI.getTableMetaClient().createNewInstant( + HoodieInstant.State.REQUESTED, instant.getAction(), instant.requestedTime()); HoodieRollbackPlan metadata = TimelineMetadataUtils .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class); return metadata.getInstantToRollback().getCommitTime(); @@ -325,18 +332,18 @@ private Map> getRolledBackInstantInfo(HoodieTimeline timeli rollbackInstants.forEach(rollbackInstant -> { try { if (rollbackInstant.isInflight()) { - HoodieInstant instantToUse = new HoodieInstant( - HoodieInstant.State.REQUESTED, rollbackInstant.getAction(), rollbackInstant.getTimestamp()); + HoodieInstant instantToUse = HoodieCLI.getTableMetaClient().createNewInstant( + HoodieInstant.State.REQUESTED, rollbackInstant.getAction(), rollbackInstant.requestedTime()); HoodieRollbackPlan metadata = TimelineMetadataUtils .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class); rollbackInfoMap.computeIfAbsent(metadata.getInstantToRollback().getCommitTime(), k -> new ArrayList<>()) - .add(rollbackInstant.getTimestamp()); + .add(rollbackInstant.requestedTime()); } else { HoodieRollbackMetadata metadata = TimelineMetadataUtils .deserializeAvroMetadata(timeline.getInstantDetails(rollbackInstant).get(), HoodieRollbackMetadata.class); metadata.getCommitsRollback().forEach(instant -> { rollbackInfoMap.computeIfAbsent(instant, k -> new ArrayList<>()) - .add(rollbackInstant.getTimestamp()); + .add(rollbackInstant.requestedTime()); }); } } catch (IOException e) { @@ -355,7 +362,7 @@ private String getRollbackInfoString(Option instant, if (HoodieTimeline.ROLLBACK_ACTION.equalsIgnoreCase(instant.get().getAction())) { rollbackInfoString = "\nRolls back\n" + getInstantToRollback(timeline, instant.get()); } else { - String instantTimestamp = instant.get().getTimestamp(); + String instantTimestamp = instant.get().requestedTime(); if (rollbackInfoMap.containsKey(instantTimestamp)) { rollbackInfoString = "\nRolled back by\n" + String.join(",\n", rollbackInfoMap.get(instantTimestamp)); } @@ -368,8 +375,12 @@ static class HoodieInstantWithModTime extends HoodieInstant { private final long modificationTimeMs; - public HoodieInstantWithModTime(StoragePathInfo pathInfo) { - super(pathInfo); + public HoodieInstantWithModTime(StoragePathInfo pathInfo, InstantGenerator instantGenerator, InstantComparator instantComparator) { + this(instantGenerator.createNewInstant(pathInfo), pathInfo, instantComparator.requestedTimeOrderedComparator()); + } + + public HoodieInstantWithModTime(HoodieInstant instant, StoragePathInfo pathInfo, Comparator comparator) { + super(instant.getState(), instant.getAction(), instant.requestedTime(), instant.getCompletionTime(), comparator); this.modificationTimeMs = pathInfo.getModificationTime(); } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CLIUtils.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CLIUtils.java index f04418e1898d..524778f8994b 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CLIUtils.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CLIUtils.java @@ -21,9 +21,10 @@ import org.apache.hudi.cli.HoodieCLI; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; +import org.apache.hudi.common.table.timeline.BaseHoodieTimeline; import static org.apache.hudi.cli.utils.CommitUtil.getTimeDaysAgo; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; @@ -35,14 +36,14 @@ */ public class CLIUtils { /** - * Gets a {@link HoodieDefaultTimeline} instance containing the instants in the specified range. + * Gets a {@link HoodieTimeline} instance containing the instants in the specified range. * * @param startTs Start instant time. * @param endTs End instant time. * @param includeArchivedTimeline Whether to include intants from the archived timeline. - * @return a {@link HoodieDefaultTimeline} instance containing the instants in the specified range. + * @return a {@link BaseHoodieTimeline} instance containing the instants in the specified range. */ - public static HoodieDefaultTimeline getTimelineInRange(String startTs, String endTs, boolean includeArchivedTimeline) { + public static HoodieTimeline getTimelineInRange(String startTs, String endTs, boolean includeArchivedTimeline) { if (isNullOrEmpty(startTs)) { startTs = getTimeDaysAgo(10); } @@ -56,7 +57,7 @@ public static HoodieDefaultTimeline getTimelineInRange(String startTs, String en if (includeArchivedTimeline) { HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); archivedTimeline.loadInstantDetailsInMemory(startTs, endTs); - return archivedTimeline.findInstantsInRange(startTs, endTs).mergeTimeline(activeTimeline); + return ((HoodieTimeline)archivedTimeline.findInstantsInRange(startTs, endTs)).mergeTimeline(activeTimeline); } return activeTimeline; } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java index 12322617fb2d..cb1b4b42172a 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/utils/CommitUtil.java @@ -20,9 +20,9 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import java.io.IOException; import java.time.ZonedDateTime; @@ -38,8 +38,10 @@ public static long countNewRecords(HoodieTableMetaClient metaClient, List extraCommitMetadata = Collections.singletonMap(HoodieCommitMetadata.SCHEMA_KEY, HoodieTestTable.PHONY_TABLE_SCHEMA); @@ -146,7 +147,7 @@ public void testShowCleans() throws Exception { // EarliestCommandRetained should be 102, since hoodie.clean.commits.retained=2 // Total Time Taken need read from metadata - rows.add(new Comparable[] {clean.getTimestamp(), "102", "2", getLatestCleanTimeTakenInMillis().toString()}); + rows.add(new Comparable[] {clean.requestedTime(), "102", "2", getLatestCleanTimeTakenInMillis().toString()}); String expected = HoodiePrintHelper.print(header, new HashMap<>(), "", false, -1, false, rows); expected = removeNonWordAndStripSpace(expected); @@ -169,7 +170,7 @@ public void testShowCleanPartitions() { HoodieInstant clean = metaClient.getActiveTimeline().reload().getCleanerTimeline().getInstantsAsStream().findFirst().get(); - Object result = shell.evaluate(() -> "clean showpartitions --clean " + clean.getTimestamp()); + Object result = shell.evaluate(() -> "clean showpartitions --clean " + clean.requestedTime()); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); TableHeader header = new TableHeader().addTableHeaderField(HoodieTableHeaderFields.HEADER_PARTITION_PATH) 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 b89687c50679..fb4c7804f676 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 @@ -27,6 +27,7 @@ import org.apache.hudi.cli.testutils.HoodieTestReplaceCommitMetadataGenerator; import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; @@ -68,6 +69,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createCompactionCommitInMetadataTable; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -128,24 +130,24 @@ private LinkedHashMap generateData() throws Exception { private LinkedHashMap generateMixedData() throws Exception { // generate data and metadata LinkedHashMap replaceCommitData = new LinkedHashMap<>(); - replaceCommitData.put(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, + replaceCommitData.put(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, "103", InProcessTimeGenerator.createNewInstantTime()), new Integer[] {15, 10}); LinkedHashMap commitData = new LinkedHashMap<>(); - commitData.put(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, + commitData.put(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "102", InProcessTimeGenerator.createNewInstantTime()), new Integer[] {15, 10}); - commitData.put(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, + commitData.put(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "101", InProcessTimeGenerator.createNewInstantTime()), new Integer[] {20, 10}); for (Map.Entry entry : commitData.entrySet()) { - String key = entry.getKey().getTimestamp(); + String key = entry.getKey().requestedTime(); Integer[] value = entry.getValue(); HoodieTestCommitMetadataGenerator.createCommitFileWithMetadata(tablePath1, key, storageConf(), Option.of(value[0]), Option.of(value[1])); } for (Map.Entry entry : replaceCommitData.entrySet()) { - String key = entry.getKey().getTimestamp(); + String key = entry.getKey().requestedTime(); Integer[] value = entry.getValue(); HoodieTestReplaceCommitMetadataGenerator.createReplaceCommitFileWithMetadata(tablePath1, key, Option.of(value[0]), Option.of(value[1]), metaClient); @@ -309,7 +311,7 @@ private Map generateDataAndArchive(boolean enableMetadataTabl // archive metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()); HoodieSparkTable table = HoodieSparkTable.create(cfg, context(), metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context()); return data; } @@ -350,7 +352,7 @@ public void testShowArchivedCommitsWithMultiCommitsFile(boolean enableMetadataTa HoodieSparkTable table = HoodieSparkTable.create(cfg, context(), metaClient); // need to create multi archive files - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context()); } @@ -413,7 +415,7 @@ public void testShowCommitPartitionsWithReplaceCommits() throws Exception { for (HoodieInstant commitInstant : data.keySet()) { Object result = shell.evaluate(() -> - String.format("commit showpartitions --commit %s", commitInstant.getTimestamp())); + String.format("commit showpartitions --commit %s", commitInstant.requestedTime())); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); @@ -486,7 +488,7 @@ public void testShowCommitFilesWithReplaceCommits() throws Exception { Map data = generateMixedData(); for (HoodieInstant commitInstant : data.keySet()) { - Object result = shell.evaluate(() -> String.format("commit showfiles --commit %s", commitInstant.getTimestamp())); + Object result = shell.evaluate(() -> String.format("commit showfiles --commit %s", commitInstant.requestedTime())); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); Integer[] value = data.get(commitInstant); @@ -537,7 +539,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) - .getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .getInstantsAsStream().map(HoodieInstant::requestedTime).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/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index c3c4a4c15c59..2a2c9179306f 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -25,12 +25,12 @@ import org.apache.hudi.cli.functional.CLIFunctionalTestHarness; import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; @@ -61,6 +61,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -157,7 +158,7 @@ private void generateCompactionInstances() throws IOException { // Create six commits Arrays.asList("001", "003", "005", "007").forEach(timestamp -> { activeTimeline.transitionCompactionInflightToComplete(true, - new HoodieInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, timestamp), Option.empty()); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, timestamp), Option.empty()); }); // Simulate a compaction commit in metadata table timeline // so the archival in data table can happen @@ -176,7 +177,7 @@ private void generateArchive() throws IOException { // archive HoodieTableMetaClient metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()); HoodieSparkTable table = HoodieSparkTable.create(cfg, context(), metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context()); } @@ -219,7 +220,7 @@ public void testCompactionShowArchived() throws IOException { // get compaction plan before compaction HoodieCompactionPlan plan = TimelineMetadataUtils.deserializeCompactionPlan( HoodieCLI.getTableMetaClient().reloadActiveTimeline().readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(instance)).get()); + INSTANT_GENERATOR.getCompactionRequestedInstant(instance)).get()); generateArchive(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java index dc297d40edc1..8804d973573e 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestDiffCommand.java @@ -59,6 +59,7 @@ import java.util.UUID; import java.util.function.Function; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -103,10 +104,10 @@ public void testDiffFile() throws Exception { commits.add(timestamp); // Requested Compaction HoodieTestCommitMetadataGenerator.createCompactionAuxiliaryMetadata(tablePath, - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp), conf); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp), conf); // Inflight Compaction HoodieTestCommitMetadataGenerator.createCompactionAuxiliaryMetadata(tablePath, - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp), conf); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp), conf); Map extraCommitMetadata = Collections.singletonMap(HoodieCommitMetadata.SCHEMA_KEY, HoodieTestTable.PHONY_TABLE_SCHEMA); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java index 34ec1b6b2335..c9424dc7cbd8 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestFileSystemViewCommand.java @@ -323,7 +323,7 @@ public void testShowLatestFileSlices() throws IOException { .addTableHeaderField(HoodieTableHeaderFields.HEADER_DELTA_FILES_UNSCHEDULED); // Test show with partition path '2016/03/15' - new TableCommand().connect(partitionedTablePath, null, false, 0, 0, 0, + new TableCommand().connect(partitionedTablePath, false, 0, 0, 0, "WAIT_TO_ADJUST_SKEW", 200L, false); Object partitionedTable = shell.evaluate(() -> "show fsview latest --partitionPath " + partitionPath); assertTrue(ShellEvaluationResultUtil.isSuccess(partitionedTable)); @@ -337,7 +337,7 @@ public void testShowLatestFileSlices() throws IOException { assertEquals(partitionedExpected, partitionedResults); // Test show for non-partitioned table - new TableCommand().connect(nonpartitionedTablePath, null, false, 0, 0, 0, + new TableCommand().connect(nonpartitionedTablePath, false, 0, 0, 0, "WAIT_TO_ADJUST_SKEW", 200L, false); Object nonpartitionedTable = shell.evaluate(() -> "show fsview latest"); assertTrue(ShellEvaluationResultUtil.isSuccess(nonpartitionedTable)); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java index 573cb33ee3a1..634acda09e05 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestMetadataCommand.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.keygen.SimpleKeyGenerator; @@ -73,7 +72,6 @@ public void testMetadataDelete() throws Exception { .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName("org.apache.hudi.common.model.HoodieAvroPayload") - .setTimelineLayoutVersion(TimelineLayoutVersion.VERSION_1) .setPartitionFields("partition_path") .setRecordKeyFields("_row_key") .setKeyGeneratorClassProp(SimpleKeyGenerator.class.getCanonicalName()) @@ -97,7 +95,7 @@ public void testMetadataDelete() throws Exception { HoodieTableMetaClient metaClient = createMetaClient(jsc(), tablePath); assertFalse(metaClient.getTableConfig().getMetadataPartitions().isEmpty()); - new TableCommand().connect(tablePath, null, false, 0, 0, 0, + new TableCommand().connect(tablePath, false, 0, 0, 0, "WAIT_TO_ADJUST_SKEW", 200L, false); Object result = shell.evaluate(() -> "metadata delete"); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); 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 e4e8da5dd4f2..9a635f77ffa1 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 @@ -285,7 +285,7 @@ public void testShowFailedCommits() { HoodieTestCommitMetadataGenerator.createCommitFile(tablePath, timestamp, conf); } - metaClient.getActiveTimeline().getInstantsAsStream().filter(hoodieInstant -> Integer.parseInt(hoodieInstant.getTimestamp()) % 4 == 0).forEach(hoodieInstant -> { + metaClient.getActiveTimeline().getInstantsAsStream().filter(hoodieInstant -> Integer.parseInt(hoodieInstant.requestedTime()) % 4 == 0).forEach(hoodieInstant -> { metaClient.getActiveTimeline().deleteInstantFileIfExists(hoodieInstant); if (hoodieInstant.isCompleted()) { metaClient.getActiveTimeline().createCompleteInstant(hoodieInstant); @@ -323,7 +323,6 @@ public void testRepairDeprecatedPartition() throws IOException { .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName("org.apache.hudi.common.model.HoodieAvroPayload") - .setTimelineLayoutVersion(TimelineLayoutVersion.VERSION_1) .setPartitionFields("partition_path") .setRecordKeyFields("_row_key") .setKeyGeneratorClassProp(SimpleKeyGenerator.class.getCanonicalName()) @@ -391,7 +390,6 @@ public void testRenamePartition() throws IOException { .setTableName(tableName()) .setArchiveLogFolder(HoodieTableConfig.ARCHIVELOG_FOLDER.defaultValue()) .setPayloadClassName("org.apache.hudi.common.model.HoodieAvroPayload") - .setTimelineLayoutVersion(TimelineLayoutVersion.VERSION_1) .setPartitionFields("partition_path") .setRecordKeyFields("_row_key") .setKeyGeneratorClassProp(SimpleKeyGenerator.class.getCanonicalName()) diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java index 0e8a9f0f218c..77930b929a22 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestRestoresCommand.java @@ -177,7 +177,7 @@ public void testShowRestore() throws IOException { HoodieInstant instant = restores.findFirst().orElse(null); assertNotNull(instant, "The instant can not be null."); - Object result = shell.evaluate(() -> "show restore --instant " + instant.getTimestamp()); + Object result = shell.evaluate(() -> "show restore --instant " + instant.requestedTime()); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); // get metadata of instant 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 09272bb38007..5641c2c4aa05 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 @@ -177,7 +177,7 @@ public void testShowRollback() throws IOException { HoodieInstant instant = rollback.findFirst().orElse(null); assertNotNull(instant, "The instant can not be null."); - Object result = shell.evaluate(() -> "show rollback --instant " + instant.getTimestamp()); + Object result = shell.evaluate(() -> "show rollback --instant " + instant.requestedTime()); assertTrue(ShellEvaluationResultUtil.isSuccess(result)); List rows = new ArrayList<>(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java index b968d2c1424d..55dc06ab90da 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestTableCommand.java @@ -32,7 +32,6 @@ import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimeGeneratorType; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.storage.StoragePath; @@ -124,7 +123,6 @@ public void testConnectTable() { // Check default values assertFalse(conf.isConsistencyCheckEnabled()); - assertEquals(new Integer(1), HoodieCLI.layoutVersion.getVersion()); assertEquals(TimeGeneratorType.valueOf("WAIT_TO_ADJUST_SKEW"), timeGeneratorConfig.getTimeGeneratorType()); } @@ -239,7 +237,7 @@ public void testFetchTableSchema() throws Exception { HoodieCLI.conf = storageConf(); new TableCommand().createTable( tablePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); String schemaStr = "{\n" diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java index 271885df9bce..9ce5b587d9fc 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestBootstrapCommand.java @@ -24,7 +24,6 @@ import org.apache.hudi.cli.testutils.HoodieCLIIntegrationTestBase; import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.functional.TestBootstrap; import org.apache.hudi.storage.StoragePath; @@ -91,7 +90,7 @@ public void testBootstrapRunCommand() throws IOException { assertTrue(ShellEvaluationResultUtil.isSuccess(resultForBootstrapRun)); // Connect & check Hudi table exist - new TableCommand().connect(tablePath, TimelineLayoutVersion.VERSION_1, false, 2000, 300000, 7, + new TableCommand().connect(tablePath, false, 2000, 300000, 7, "WAIT_TO_ADJUST_SKEW", 200L, false); metaClient = HoodieCLI.getTableMetaClient(); assertEquals(1, metaClient.getActiveTimeline().getCommitsTimeline().countInstants(), "Should have 1 commit."); 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 d356da22fe68..0414924c792f 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 @@ -31,7 +31,6 @@ import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieIndexConfig; @@ -78,7 +77,7 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); @@ -117,7 +116,7 @@ public void testClustering() throws IOException { // get clustering instance HoodieActiveTimeline timeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); Option instanceOpt = - timeline.filterPendingClusteringTimeline().firstInstant().map(HoodieInstant::getTimestamp); + timeline.filterPendingClusteringTimeline().firstInstant().map(HoodieInstant::requestedTime); assertTrue(instanceOpt.isPresent(), "Must have pending clustering."); final String instance = instanceOpt.get(); @@ -133,12 +132,12 @@ public void testClustering() throws IOException { // assert clustering complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() .filterCompletedInstants().getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), + .map(HoodieInstant::requestedTime).collect(Collectors.toList()).contains(instance), "Pending clustering must be completed"); assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() .getCompletedReplaceTimeline().getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), + .map(HoodieInstant::requestedTime).collect(Collectors.toList()).contains(instance), "Pending clustering must be completed"); } @@ -161,7 +160,7 @@ public void testClusteringScheduleAndExecute() throws IOException { // assert clustering complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() .getCompletedReplaceTimeline().getInstantsAsStream() - .map(HoodieInstant::getTimestamp).count() > 0, + .map(HoodieInstant::requestedTime).count() > 0, "Completed clustering couldn't be 0"); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java index 80d9bc9fa4a5..06c12c93f542 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestCommitsCommand.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -77,7 +76,7 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); 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 d15d444c76bf..53e695f10737 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 @@ -34,7 +34,6 @@ import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.CompactionTestUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.CompactionUtils; @@ -84,7 +83,7 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( basePath, tableName, HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); initMetaClient(); @@ -136,7 +135,7 @@ public void testCompact() throws IOException { // assert compaction complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() .filterCompletedInstants().getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).contains(instance), + .map(HoodieInstant::requestedTime).collect(Collectors.toList()).contains(instance), "Pending compaction must be completed"); } @@ -164,7 +163,7 @@ public void testCompactScheduleAndExecute() throws IOException { // assert compaction complete assertTrue(HoodieCLI.getTableMetaClient().getActiveTimeline().reload() .filterCompletedInstants().getInstantsAsStream() - .map(HoodieInstant::getTimestamp).count() > 0, + .map(HoodieInstant::requestedTime).count() > 0, "Completed compaction couldn't be 0"); } @@ -269,7 +268,7 @@ private String prepareScheduleCompaction() { // get compaction instance HoodieActiveTimeline timeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); Option instance = - timeline.filterPendingCompactionTimeline().firstInstant().map(HoodieInstant::getTimestamp); + timeline.filterPendingCompactionTimeline().firstInstant().map(HoodieInstant::requestedTime); assertTrue(instance.isPresent(), "Must have pending compaction."); return instance.get(); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java index 4cfc6ca13d1c..40d8cc04f58b 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestHDFSParquetImportCommand.java @@ -24,7 +24,6 @@ import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.storage.StoragePath; import org.apache.hudi.testutils.HoodieClientTestUtils; @@ -113,7 +112,7 @@ public void testConvertWithInsert() throws IOException { assertTrue(Files.exists(Paths.get(metaPath)), "Hoodie table not exist."); // Load meta data - new TableCommand().connect(targetPath.toString(), TimelineLayoutVersion.VERSION_1, false, 2000, 300000, 7, + new TableCommand().connect(targetPath.toString(), false, 2000, 300000, 7, "WAIT_TO_ADJUST_SKEW", 200L, false); metaClient = HoodieCLI.getTableMetaClient(); @@ -138,7 +137,7 @@ public void testConvertWithUpsert() throws IOException, ParseException { dataImporter.dataImport(jsc, 0); // Load meta data - new TableCommand().connect(targetPath.toString(), TimelineLayoutVersion.VERSION_1, false, 2000, 300000, 7, + new TableCommand().connect(targetPath.toString(), false, 2000, 300000, 7, "WAIT_TO_ADJUST_SKEW", 200L, false); metaClient = HoodieCLI.getTableMetaClient(); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java index d03ce6a0e2ab..c254cbcc18a2 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestMarkersCommand.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableVersion; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.storage.StoragePath; @@ -60,7 +59,7 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java index ee223321e822..8d2107316918 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestRepairsCommand.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.SchemaTestUtil; @@ -102,7 +101,7 @@ public void init() throws Exception { // Create cow table and connect new TableCommand().createTable( cowTablePath, "cow_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable cowTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); @@ -126,7 +125,7 @@ public void init() throws Exception { // Create mor table and connect new TableCommand().createTable( morTablePath, "mor_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable morTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); @@ -149,7 +148,7 @@ public void init() throws Exception { // Create cow table and connect new TableCommand().createTable( cowNonPartitionedTablePath, "cow_table_non_partitioned", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieSparkWriteableTestTable cowNonPartitionedTable = HoodieSparkWriteableTestTable.of(HoodieCLI.getTableMetaClient(), schema); @@ -335,7 +334,7 @@ public void testDeduplicateWithReal(HoodieTableType tableType) throws IOExceptio } private void connectTableAndReloadMetaClient(String tablePath) throws IOException { - new TableCommand().connect(tablePath, TimelineLayoutVersion.VERSION_1, false, 0, 0, 0, + new TableCommand().connect(tablePath, false, 0, 0, 0, "WAIT_TO_ADJUST_SKEW", 200L, false); metaClient = HoodieTableMetaClient.reload(HoodieCLI.getTableMetaClient()); } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java index 4912db60d3d6..b0fbb9f8718a 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestSavepointsCommand.java @@ -29,7 +29,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -46,6 +45,7 @@ import java.io.IOException; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -72,7 +72,7 @@ public void init() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, HoodieTableVersion.current().versionCode(), + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); } @@ -140,7 +140,7 @@ public void testRollbackToSavepoint() throws IOException { // 103 instant had rollback assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant( - new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "103"))); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "103"))); } /** @@ -185,9 +185,9 @@ public void testRollbackToSavepointWithMetadataTableEnable() throws Exception { // 103 and 104 instant had rollback assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant( - new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "103"))); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "103"))); assertFalse(timeline.getCommitAndReplaceTimeline().containsInstant( - new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", "104"))); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "104"))); } /** @@ -226,6 +226,6 @@ public void testDeleteSavepoint() throws IOException { assertEquals(1, timeline.getSavePointTimeline().countInstants(), "There should 1 instants."); // after delete, 100 instant should not exist. - assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepoint1))); + assertFalse(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepoint1))); } } diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java index 6bdbc2730280..2ae820facf8c 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/integ/ITTestTableCommand.java @@ -27,10 +27,10 @@ import org.apache.hudi.common.model.HoodieAvroPayload; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieIndexConfig; @@ -77,7 +77,7 @@ public void testChangeTableCOW2MOR() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.COPY_ON_WRITE.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); HoodieTestDataGenerator.createCommitFile( tablePath, "100", HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration())); @@ -95,7 +95,7 @@ public void testChangeTableMOR2COW() throws IOException { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); Object result = shell.evaluate(() -> "table change-table-type --target-type COW"); @@ -110,7 +110,7 @@ public void testChangeTableMOR2COW_withPendingCompactions() throws Exception { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); generateCommits(); // schedule a compaction @@ -142,7 +142,7 @@ public void testChangeTableMOR2COW_withFullCompaction() throws Exception { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); generateCommits(); Object result = shell.evaluate(() -> "table change-table-type --target-type COW"); @@ -167,7 +167,7 @@ public void testChangeTableMOR2COW_withoutCompaction() throws Exception { // Create table and connect new TableCommand().createTable( tablePath, "test_table", HoodieTableType.MERGE_ON_READ.name(), - "", TimelineLayoutVersion.VERSION_1, "org.apache.hudi.common.model.HoodieAvroPayload"); + "", HoodieTableVersion.current().versionCode(), "org.apache.hudi.common.model.HoodieAvroPayload"); generateCommits(); Object result = shell.evaluate(() -> "table change-table-type --target-type COW --enable-compaction false"); diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java index 5a6b4295a252..141be814d961 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/testutils/HoodieTestCommitMetadataGenerator.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.InProcessTimeGenerator; @@ -41,6 +40,8 @@ import java.util.Map; import java.util.UUID; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.apache.hudi.common.testutils.FileCreateUtils.baseFileName; import static org.apache.hudi.common.util.CollectionUtils.createImmutableList; @@ -88,13 +89,13 @@ public static void createCommitFileWithMetadata(String basePath, String commitTi public static void createCommitFileWithMetadata(String basePath, String commitTime, StorageConfiguration configuration, String fileId1, String fileId2, Option writes, Option updates, Map extraMetadata) throws Exception { List commitFileNames = Arrays.asList( - HoodieTimeline.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - HoodieTimeline.makeInflightCommitFileName(commitTime), - HoodieTimeline.makeRequestedCommitFileName(commitTime)); + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + INSTANT_FILE_NAME_GENERATOR.makeInflightCommitFileName(commitTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedCommitFileName(commitTime)); for (String name : commitFileNames) { HoodieCommitMetadata commitMetadata = generateCommitMetadata(basePath, commitTime, fileId1, fileId2, writes, updates, extraMetadata, true); - createFileWithMetadata(basePath, configuration, name, serializeCommitMetadata(commitMetadata).get()); + createFileWithMetadata(basePath, configuration, name, serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } } @@ -103,13 +104,13 @@ public static void createCommitFileWithMetadata(String basePath, String commitTi Option updates, Map extraMetadata, boolean setDefaultFileId) throws Exception { List commitFileNames = Arrays.asList( - HoodieTimeline.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - HoodieTimeline.makeInflightCommitFileName(commitTime), - HoodieTimeline.makeRequestedCommitFileName(commitTime)); + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + INSTANT_FILE_NAME_GENERATOR.makeInflightCommitFileName(commitTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedCommitFileName(commitTime)); for (String name : commitFileNames) { HoodieCommitMetadata commitMetadata = generateCommitMetadata(basePath, commitTime, fileId1, fileId2, writes, updates, extraMetadata, setDefaultFileId); - createFileWithMetadata(basePath, configuration, name, serializeCommitMetadata(commitMetadata).get()); + createFileWithMetadata(basePath, configuration, name, serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } } diff --git a/hudi-cli/src/test/resources/table-config.properties b/hudi-cli/src/test/resources/table-config.properties index d74c0444a572..97087c10a321 100644 --- a/hudi-cli/src/test/resources/table-config.properties +++ b/hudi-cli/src/test/resources/table-config.properties @@ -18,4 +18,4 @@ hoodie.table.name=test_table hoodie.table.type=COPY_ON_WRITE hoodie.archivelog.folder=archive -hoodie.timeline.layout.version=1 +hoodie.timeline.layout.version=2 diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java index 763150d5429f..7b3884235010 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncClusteringService.java @@ -23,7 +23,6 @@ import org.apache.hudi.client.BaseHoodieWriteClient; import org.apache.hudi.common.engine.EngineProperty; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -76,7 +75,7 @@ protected Pair startService() { LOG.info("Setting pool name for clustering to " + CLUSTERING_POOL_NAME); context.setProperty(EngineProperty.CLUSTERING_POOL_NAME, CLUSTERING_POOL_NAME); while (!isShutdownRequested()) { - final HoodieInstant instant = fetchNextAsyncServiceInstant(); + final String instant = fetchNextAsyncServiceInstant(); if (null != instant) { LOG.info("Starting clustering for instant " + instant); clusteringClient.cluster(instant); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java index c3728a19b2e3..1d1b9421b596 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/AsyncCompactService.java @@ -21,7 +21,6 @@ import org.apache.hudi.client.BaseHoodieWriteClient; import org.apache.hudi.common.engine.EngineProperty; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.CustomizedThreadFactory; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -77,12 +76,12 @@ protected Pair startService() { context.setProperty(EngineProperty.COMPACTION_POOL_NAME, COMPACT_POOL_NAME); while (!isShutdownRequested()) { - final HoodieInstant instant = fetchNextAsyncServiceInstant(); + final String instantTime = fetchNextAsyncServiceInstant(); - if (null != instant) { - LOG.info("Starting Compaction for instant " + instant); - compactor.compact(instant); - LOG.info("Finished Compaction for instant " + instant); + if (null != instantTime) { + LOG.info("Starting Compaction for instant " + instantTime); + compactor.compact(instantTime); + LOG.info("Finished Compaction for instant " + instantTime); } } LOG.info("Compactor shutting down properly!!"); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java index f022e7104568..989babfdcb7a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/async/HoodieAsyncService.java @@ -18,7 +18,6 @@ package org.apache.hudi.async; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.collection.Pair; import org.slf4j.Logger; @@ -58,7 +57,7 @@ public abstract class HoodieAsyncService implements Serializable { // Run in daemon mode private final boolean runInDaemonMode; // Queue to hold pending compaction/clustering instants - private transient BlockingQueue pendingInstants = new LinkedBlockingQueue<>(); + private transient BlockingQueue pendingInstants = new LinkedBlockingQueue<>(); // Mutex lock for synchronized access to pendingInstants queue private transient ReentrantLock queueLock = new ReentrantLock(); // Condition instance to use with the queueLock @@ -197,23 +196,23 @@ public void waitTillPendingAsyncServiceInstantsReducesTo(int numPending) throws /** * Enqueues new pending table service instant. - * @param instant {@link HoodieInstant} to enqueue. + * @param instantTime {@link String} to enqueue. */ - public void enqueuePendingAsyncServiceInstant(HoodieInstant instant) { - LOG.info("Enqueuing new pending table service instant: " + instant.getTimestamp()); - pendingInstants.add(instant); + public void enqueuePendingAsyncServiceInstant(String instantTime) { + LOG.info("Enqueuing new pending table service instant: " + instantTime); + pendingInstants.add(instantTime); } /** * Fetch next pending compaction/clustering instant if available. * - * @return {@link HoodieInstant} corresponding to the next pending compaction/clustering. + * @return {@link String} corresponding to the next pending compaction/clustering. * @throws InterruptedException */ - HoodieInstant fetchNextAsyncServiceInstant() throws InterruptedException { + String fetchNextAsyncServiceInstant() throws InterruptedException { LOG.info(String.format("Waiting for next instant up to %d seconds", POLLING_SECONDS)); - HoodieInstant instant = pendingInstants.poll(POLLING_SECONDS, TimeUnit.SECONDS); - if (instant != null) { + String instantTime = pendingInstants.poll(POLLING_SECONDS, TimeUnit.SECONDS); + if (instantTime != null) { try { queueLock.lock(); // Signal waiting thread @@ -222,6 +221,6 @@ HoodieInstant fetchNextAsyncServiceInstant() throws InterruptedException { queueLock.unlock(); } } - return instant; + return instantTime; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java index a3f552e640ff..8b5dcb474b20 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseClusterer.java @@ -19,8 +19,6 @@ package org.apache.hudi.client; -import org.apache.hudi.common.table.timeline.HoodieInstant; - import java.io.IOException; import java.io.Serializable; @@ -39,10 +37,10 @@ public BaseClusterer(BaseHoodieWriteClient clusteringClient) { /** * Run clustering for the instant. - * @param instant + * @param instantTime * @throws IOException */ - public abstract void cluster(HoodieInstant instant) throws IOException; + public abstract void cluster(String instantTime) throws IOException; /** * Update the write client used by async clustering. diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java index ba4d3f77fd9e..8cbcfe71f73f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseCompactor.java @@ -18,8 +18,6 @@ package org.apache.hudi.client; -import org.apache.hudi.common.table.timeline.HoodieInstant; - import java.io.IOException; import java.io.Serializable; @@ -36,7 +34,7 @@ public BaseCompactor(BaseHoodieWriteClient compactionClient) { this.compactionClient = compactionClient; } - public abstract void compact(HoodieInstant instant) throws IOException; + public abstract void compact(String instantTime) throws IOException; public void updateWriteClient(BaseHoodieWriteClient writeClient) { this.compactionClient = writeClient; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java index 55bac6942751..7651057c05f7 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieClient.java @@ -29,10 +29,9 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -186,7 +185,6 @@ protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoa .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(loadActiveTimelineOnLoad) .setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setTimeGeneratorConfig(config.getTimeGeneratorConfig()) .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setMetaserverConfig(config.getProps()).build(); @@ -198,14 +196,14 @@ protected HoodieTableMetaClient createMetaClient(boolean loadActiveTimelineOnLoa * @param milliseconds Milliseconds to add to current time while generating the new instant time. */ public String createNewInstantTime(long milliseconds) { - return HoodieActiveTimeline.createNewInstantTime(true, timeGenerator, milliseconds); + return TimelineUtils.generateInstantTime(true, timeGenerator, milliseconds); } /** * Returns next instant time in the correct format. An explicit Lock is enabled in the context. */ public String createNewInstantTime() { - return HoodieActiveTimeline.createNewInstantTime(true, timeGenerator); + return TimelineUtils.generateInstantTime(true, timeGenerator); } /** @@ -214,7 +212,7 @@ public String createNewInstantTime() { * @param shouldLock Whether to lock the context to get the instant time. */ public String createNewInstantTime(boolean shouldLock) { - return HoodieActiveTimeline.createNewInstantTime(shouldLock, timeGenerator); + return TimelineUtils.generateInstantTime(shouldLock, timeGenerator); } public Option getTimelineServer() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 368879c089ef..fa43da009ede 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -29,6 +29,7 @@ import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.TimelineArchivers; import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; @@ -39,10 +40,14 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; @@ -86,7 +91,10 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.apache.hudi.metadata.HoodieTableMetadata.isMetadataTable; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.isIndexingCommit; @@ -208,7 +216,7 @@ protected HoodieWriteMetadata logCompact(String logCompactionInstantTime, boo Option compactionInstantWithGreaterTimestamp = Option.fromJavaOptional(table.getActiveTimeline().getInstantsAsStream() .filter(hoodieInstant -> actions.contains(hoodieInstant.getAction())) - .filter(hoodieInstant -> HoodieTimeline.compareTimestamps(hoodieInstant.getTimestamp(), + .filter(hoodieInstant -> compareTimestamps(hoodieInstant.requestedTime(), GREATER_THAN, logCompactionInstantTime)) .findFirst()); if (compactionInstantWithGreaterTimestamp.isPresent()) { @@ -217,7 +225,8 @@ protected HoodieWriteMetadata logCompact(String logCompactionInstantTime, boo } HoodieTimeline pendingLogCompactionTimeline = table.getActiveTimeline().filterPendingLogCompactionTimeline(); - HoodieInstant inflightInstant = HoodieTimeline.getLogCompactionInflightInstant(logCompactionInstantTime); + InstantGenerator instantGenerator = table.getMetaClient().getInstantGenerator(); + HoodieInstant inflightInstant = instantGenerator.getLogCompactionInflightInstant(logCompactionInstantTime); if (pendingLogCompactionTimeline.containsInstant(inflightInstant)) { LOG.info("Found Log compaction inflight file. Rolling back the commit and exiting."); table.rollbackInflightLogCompaction(inflightInstant, commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); @@ -251,7 +260,7 @@ protected void runAnyPendingCompactions(HoodieTable table) { table.getActiveTimeline().getWriteTimeline().filterPendingCompactionTimeline().getInstants() .forEach(instant -> { LOG.info("Running previously failed inflight compaction at instant {}", instant); - compact(instant.getTimestamp(), true); + compact(instant.requestedTime(), true); }); } @@ -259,7 +268,7 @@ protected void runAnyPendingLogCompactions(HoodieTable table) { table.getActiveTimeline().getWriteTimeline().filterPendingLogCompactionTimeline().getInstantsAsStream() .forEach(instant -> { LOG.info("Running previously failed inflight log compaction at instant {}", instant); - logCompact(instant.getTimestamp(), true); + logCompact(instant.requestedTime(), true); }); } @@ -292,7 +301,8 @@ public Option scheduleCompaction(Option> extraMetada protected HoodieWriteMetadata compact(String compactionInstantTime, boolean shouldComplete) { HoodieTable table = createTable(config, context.getStorageConf()); HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline(); - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime); + InstantGenerator instantGenerator = table.getMetaClient().getInstantGenerator(); + HoodieInstant inflightInstant = instantGenerator.getCompactionInflightInstant(compactionInstantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { table.rollbackInflightCompaction(inflightInstant, commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); table.getMetaClient().reloadActiveTimeline(); @@ -325,7 +335,8 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName()); List writeStats = metadata.getWriteStats(); handleWriteErrors(writeStats, TableServiceType.COMPACT); - final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); + InstantGenerator instantGenerator = table.getMetaClient().getInstantGenerator(); + final HoodieInstant compactionInstant = instantGenerator.getCompactionInflightInstant(compactionCommitTime); try { this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); finalizeWrite(table, compactionCommitTime, writeStats); @@ -342,7 +353,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); - HoodieActiveTimeline.parseDateFromInstantTimeSafely(compactionCommitTime).ifPresent(parsedInstant -> + TimelineUtils.parseDateFromInstantTimeSafely(compactionCommitTime).ifPresent(parsedInstant -> metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, COMPACTION_ACTION) ); } @@ -386,7 +397,8 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, HoodieTable this.context.setJobStatus(this.getClass().getSimpleName(), "Collect log compaction write status and commit compaction"); List writeStats = metadata.getWriteStats(); handleWriteErrors(writeStats, TableServiceType.LOG_COMPACT); - final HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionCommitTime); + final HoodieInstant logCompactionInstant = table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, + logCompactionCommitTime); try { this.txnManager.beginTransaction(Option.of(logCompactionInstant), Option.empty()); preCommit(metadata); @@ -404,7 +416,7 @@ protected void completeLogCompaction(HoodieCommitMetadata metadata, HoodieTable .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (logCompactionTimer != null) { long durationInMs = metrics.getDurationInMs(logCompactionTimer.stop()); - HoodieActiveTimeline.parseDateFromInstantTimeSafely(logCompactionCommitTime).ifPresent(parsedInstant -> + TimelineUtils.parseDateFromInstantTimeSafely(logCompactionCommitTime).ifPresent(parsedInstant -> metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, HoodieActiveTimeline.LOG_COMPACTION_ACTION) ); } @@ -450,9 +462,10 @@ public boolean scheduleClusteringAtInstant(String instantTime, Option cluster(String clusteringInstant, boolean shouldComplete) { HoodieTable table = createTable(config, context.getStorageConf()); HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceOrClusteringTimeline(); - Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline()); + Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline(), + table.getMetaClient().getInstantGenerator()); if (inflightInstantOpt.isPresent()) { - if (pendingClusteringTimeline.isPendingClusteringInstant(inflightInstantOpt.get().getTimestamp())) { + if (pendingClusteringTimeline.isPendingClusteringInstant(inflightInstantOpt.get().requestedTime())) { table.rollbackInflightClustering(inflightInstantOpt.get(), commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); table.getMetaClient().reloadActiveTimeline(); } else { @@ -484,7 +497,8 @@ public HoodieWriteMetadata cluster(String clusteringInstant, boolean shouldCo public boolean purgePendingClustering(String clusteringInstant) { HoodieTable table = createTable(config, context.getStorageConf()); - Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline()); + Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant, table.getActiveTimeline(), + table.getMetaClient().getInstantGenerator()); if (inflightInstantOpt.isPresent()) { table.rollbackInflightClustering(inflightInstantOpt.get(), commitToRollback -> getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false), true); table.getMetaClient().reloadActiveTimeline(); @@ -516,7 +530,8 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, Option> writeStatuses) { List writeStats = metadata.getWriteStats(); handleWriteErrors(writeStats, TableServiceType.CLUSTER); - final HoodieInstant clusteringInstant = ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline()).get(); + final HoodieInstant clusteringInstant = ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, + table.getActiveTimeline(), table.getMetaClient().getInstantGenerator()).get(); try { this.txnManager.beginTransaction(Option.of(clusteringInstant), Option.empty()); @@ -527,12 +542,13 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, preCommit(metadata); } // Update table's metadata (table) - writeTableMetadata(table, clusteringInstant.getTimestamp(), metadata, writeStatuses.orElseGet(context::emptyHoodieData)); + writeTableMetadata(table, clusteringInstant.requestedTime(), metadata, writeStatuses.orElseGet(context::emptyHoodieData)); LOG.info("Committing Clustering {}", clusteringCommitTime); LOG.debug("Clustering {} finished with result {}", clusteringCommitTime, metadata); - ClusteringUtils.transitionClusteringOrReplaceInflightToComplete(false, clusteringInstant, serializeCommitMetadata(metadata), table.getActiveTimeline()); + ClusteringUtils.transitionClusteringOrReplaceInflightToComplete(false, clusteringInstant, + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), metadata), table.getActiveTimeline()); } catch (Exception e) { throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e); } finally { @@ -543,7 +559,7 @@ private void completeClustering(HoodieReplaceCommitMetadata metadata, .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); if (clusteringTimer != null) { long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); - HoodieActiveTimeline.parseDateFromInstantTimeSafely(clusteringCommitTime).ifPresent(parsedInstant -> + TimelineUtils.parseDateFromInstantTimeSafely(clusteringCommitTime).ifPresent(parsedInstant -> metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, HoodieActiveTimeline.CLUSTERING_ACTION) ); } @@ -617,7 +633,9 @@ protected void runTableServicesInline(HoodieTable table, HoodieCommitMetadata me public Option scheduleTableService(String instantTime, Option> extraMetadata, TableServiceType tableServiceType) { // A lock is required to guard against race conditions between an ongoing writer and scheduling a table service. - final Option inflightInstant = Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, + HoodieTableConfig tableConfig = HoodieTableConfig.loadFromHoodieProps(storage, config.getBasePath()); + InstantGenerator instantGenerator = TimelineLayout.fromVersion(tableConfig.getTableVersion().getTimelineLayoutVersion()).getInstantGenerator(); + final Option inflightInstant = Option.of(instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, tableServiceType.getAction(), instantTime)); try { this.txnManager.beginTransaction(inflightInstant, Option.empty()); @@ -731,7 +749,7 @@ protected void runAnyPendingClustering(HoodieTable table) { Option> instantPlan = ClusteringUtils.getClusteringPlan(table.getMetaClient(), instant); if (instantPlan.isPresent()) { LOG.info("Running pending clustering at instant {}", instantPlan.get().getLeft()); - cluster(instant.getTimestamp(), true); + cluster(instant.requestedTime(), true); } }); } @@ -816,7 +834,7 @@ protected void archive(HoodieTable table) { // Reload table timeline to reflect the latest commits, // there are some table services (for e.g, the cleaning) that executed right before the archiving. table.getMetaClient().reloadActiveTimeline(); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table); + HoodieTimelineArchiver archiver = TimelineArchivers.getInstance(table.getMetaClient().getTimelineLayoutVersion(), config, table); int instantsToArchive = archiver.archiveIfRequired(context, true); if (timerContext != null) { long durationMs = metrics.getDurationInMs(timerContext.stop()); @@ -835,7 +853,8 @@ protected void archive(HoodieTable table) { */ private HoodieTimeline getInflightTimelineExcludeCompactionAndClustering(HoodieTableMetaClient metaClient) { HoodieTimeline inflightTimelineExcludingCompaction = metaClient.getCommitsTimeline().filterPendingExcludingCompaction(); - return inflightTimelineExcludingCompaction.filter(instant -> !ClusteringUtils.isClusteringInstant(inflightTimelineExcludingCompaction, instant)); + return inflightTimelineExcludingCompaction.filter(instant -> !ClusteringUtils.isClusteringInstant( + inflightTimelineExcludingCompaction, instant, metaClient.getInstantGenerator())); } protected Option getPendingRollbackInfo(HoodieTableMetaClient metaClient, String commitToRollback) { @@ -885,7 +904,9 @@ protected Map> getPendingRollbackInfos String instantToRollback = rollbackPlan.getInstantToRollback().getCommitTime(); if (ignoreCompactionAndClusteringInstants) { if (!HoodieTimeline.COMPACTION_ACTION.equals(action)) { - boolean isClustering = ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), new HoodieInstant(true, action, instantToRollback)); + InstantGenerator instantGenerator = metaClient.getInstantGenerator(); + boolean isClustering = ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, action, instantToRollback), instantGenerator); if (!isClustering) { infoMap.putIfAbsent(instantToRollback, Option.of(new HoodiePendingRollbackInfo(rollbackInstant, rollbackPlan))); } @@ -929,15 +950,15 @@ private List getFailedIndexingCommitsToRollbackForMetadataTable(HoodieTa Stream inflightInstantsStream = metaClient.getCommitsTimeline() .filter(instant -> !instant.isCompleted() - && isIndexingCommit(dataIndexTimeline, instant.getTimestamp())) + && isIndexingCommit(dataIndexTimeline, instant.requestedTime())) .getInstantsAsStream(); return inflightInstantsStream.filter(instant -> { try { - return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); + return heartbeatClient.isHeartbeatExpired(instant.requestedTime()); } catch (IOException io) { throw new HoodieException("Failed to check heartbeat for instant " + instant, io); } - }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + }).map(HoodieInstant::requestedTime).collect(Collectors.toList()); } /** @@ -970,7 +991,7 @@ protected void rollbackFailedWrites(Map> entry : reverseSortedRollbackInstants.entrySet()) { if (!isMetadataTable - && HoodieTimeline.compareTimestamps(entry.getKey(), HoodieTimeline.LESSER_THAN_OR_EQUALS, + && compareTimestamps(entry.getKey(), LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { // do we need to handle failed rollback of a bootstrap rollbackFailedBootstrap(); @@ -997,7 +1018,7 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H .setConf(metaClient.getStorageConf().newInstance()) .build(); HoodieTimeline dataIndexTimeline = dataMetaClient.getActiveTimeline().filter(instant -> instant.getAction().equals(HoodieTimeline.INDEXING_ACTION)); - return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> { + return inflightInstantsStream.map(HoodieInstant::requestedTime).filter(entry -> { if (curInstantTime.isPresent()) { return !entry.equals(curInstantTime.get()); } else { @@ -1005,7 +1026,7 @@ protected List getInstantsToRollback(HoodieTableMetaClient metaClient, H } }).collect(Collectors.toList()); } - return inflightInstantsStream.map(HoodieInstant::getTimestamp).filter(entry -> { + return inflightInstantsStream.map(HoodieInstant::requestedTime).filter(entry -> { if (curInstantTime.isPresent()) { return !entry.equals(curInstantTime.get()); } else { @@ -1027,11 +1048,11 @@ private List getInstantsToRollbackForLazyCleanPolicy(HoodieTableMetaClie List expiredInstants = inflightInstantsStream.filter(instant -> { try { // An instant transformed from inflight to completed have no heartbeat file and will be detected as expired instant here - return heartbeatClient.isHeartbeatExpired(instant.getTimestamp()); + return heartbeatClient.isHeartbeatExpired(instant.requestedTime()); } catch (IOException io) { throw new HoodieException("Failed to check heartbeat for instant " + instant, io); } - }).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + }).map(HoodieInstant::requestedTime).collect(Collectors.toList()); if (!expiredInstants.isEmpty()) { // Only return instants that haven't been completed by other writers @@ -1054,7 +1075,7 @@ private List getInstantsToRollbackForLazyCleanPolicy(HoodieTableMetaClie @Deprecated public boolean rollback(final String commitInstantTime, Option pendingRollbackInfo, boolean skipLocking, boolean skipVersionCheck) throws HoodieRollbackException { - final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().getTimestamp()) + final String rollbackInstantTime = pendingRollbackInfo.map(entry -> entry.getRollbackInstant().requestedTime()) .orElseGet(() -> createNewInstantTime(!skipLocking)); return rollback(commitInstantTime, pendingRollbackInfo, rollbackInstantTime, skipLocking, skipVersionCheck); } @@ -1075,7 +1096,7 @@ public boolean rollback(final String commitInstantTime, Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream() - .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) + .filter(instant -> EQUALS.test(instant.requestedTime(), commitInstantTime)) .findFirst()); if (commitInstantOpt.isPresent() || pendingRollbackInfo.isPresent()) { LOG.info(String.format("Scheduling Rollback at instant time : %s " @@ -1093,8 +1114,8 @@ public boolean rollback(final String commitInstantTime, Option instant = Option.fromJavaOptional( - inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp).findFirst()); - if (instant.isPresent() && HoodieTimeline.compareTimestamps(instant.get(), HoodieTimeline.LESSER_THAN_OR_EQUALS, + inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::requestedTime).findFirst()); + if (instant.isPresent() && compareTimestamps(instant.get(), LESSER_THAN_OR_EQUALS, HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS)) { LOG.info("Found pending bootstrap instants. Rolling them back"); table.rollbackBootstrap(context, createNewInstantTime()); 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 3e258d892b64..07df2e8b7f11 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 @@ -52,6 +52,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; @@ -112,6 +113,7 @@ import static org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName; import static org.apache.hudi.common.model.HoodieCommitMetadata.SCHEMA_KEY; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; @@ -232,7 +234,7 @@ public boolean commitStats(String instantTime, HoodieData writeStat HoodieTable table = createTable(config); HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getWriteSchema(), commitActionType); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, commitActionType, instantTime); + HoodieInstant inflightInstant = table.getMetaClient().createNewInstant(State.INFLIGHT, commitActionType, instantTime); HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config); this.txnManager.beginTransaction(Option.of(inflightInstant), lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty()); @@ -292,8 +294,8 @@ protected void commit(HoodieTable table, String commitActionType, String instant } // update Metadata table writeTableMetadata(table, instantTime, metadata, writeStatuses); - activeTimeline.saveAsComplete(false, new HoodieInstant(true, commitActionType, instantTime), - serializeCommitMetadata(metadata)); + activeTimeline.saveAsComplete(false, table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, commitActionType, instantTime), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), metadata)); } // Save internal schema @@ -357,7 +359,7 @@ void emitCommitMetrics(String instantTime, HoodieCommitMetadata metadata, String long durationInMs = metrics.getDurationInMs(writeTimer.stop()); // instantTime could be a non-standard value, so use `parseDateFromInstantTimeSafely` // e.g. INIT_INSTANT_TS, METADATA_BOOTSTRAP_INSTANT_TS and FULL_BOOTSTRAP_INSTANT_TS in HoodieTimeline - HoodieActiveTimeline.parseDateFromInstantTimeSafely(instantTime).ifPresent(parsedInstant -> + TimelineUtils.parseDateFromInstantTimeSafely(instantTime).ifPresent(parsedInstant -> metrics.updateCommitMetrics(parsedInstant.getTime(), durationInMs, metadata, actionType) ); writeTimer = null; @@ -643,7 +645,7 @@ public void savepoint(String user, String comment) { throw new HoodieSavepointException("Could not savepoint. Commit timeline is empty"); } - String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().getTimestamp(); + String latestCommit = table.getCompletedCommitsTimeline().lastInstant().get().requestedTime(); LOG.info("Savepointing latest commit " + latestCommit); savepoint(latestCommit, user, comment); } @@ -676,7 +678,7 @@ public void deleteSavepoint() { throw new HoodieSavepointException("Could not delete savepoint. Savepoint timeline is empty"); } - String savepointTime = savePointTimeline.lastInstant().get().getTimestamp(); + String savepointTime = savePointTimeline.lastInstant().get().requestedTime(); LOG.info("Deleting latest savepoint time " + savepointTime); deleteSavepoint(savepointTime); } @@ -702,7 +704,7 @@ public void restoreToSavepoint() { throw new HoodieSavepointException("Could not restore to savepoint. Savepoint timeline is empty"); } - String savepointTime = savePointTimeline.lastInstant().get().getTimestamp(); + String savepointTime = savePointTimeline.lastInstant().get().requestedTime(); LOG.info("Restoring to latest savepoint time " + savepointTime); restoreToSavepoint(savepointTime); } @@ -729,9 +731,9 @@ public void restoreToSavepoint(String savepointTime) { Option oldestMdtCompaction = mdtMetaClient.getCommitTimeline().filterCompletedInstants().firstInstant(); boolean deleteMDT = false; if (oldestMdtCompaction.isPresent()) { - if (HoodieTimeline.LESSER_THAN_OR_EQUALS.test(savepointTime, oldestMdtCompaction.get().getTimestamp())) { + if (LESSER_THAN_OR_EQUALS.test(savepointTime, oldestMdtCompaction.get().requestedTime())) { LOG.warn(String.format("Deleting MDT during restore to %s as the savepoint is older than oldest compaction %s on MDT", - savepointTime, oldestMdtCompaction.get().getTimestamp())); + savepointTime, oldestMdtCompaction.get().requestedTime())); deleteMDT = true; } } @@ -739,10 +741,10 @@ public void restoreToSavepoint(String savepointTime) { // The instant required to sync rollback to MDT has been archived and the mdt syncing will be failed // So that we need to delete the whole MDT here. if (!deleteMDT) { - HoodieInstant syncedInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, savepointTime); - if (mdtMetaClient.getCommitsTimeline().isBeforeTimelineStarts(syncedInstant.getTimestamp())) { + HoodieInstant syncedInstant = mdtMetaClient.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, savepointTime); + if (mdtMetaClient.getCommitsTimeline().isBeforeTimelineStarts(syncedInstant.requestedTime())) { LOG.warn(String.format("Deleting MDT during restore to %s as the savepoint is older than the MDT timeline %s", - savepointTime, mdtMetaClient.getCommitsTimeline().firstInstant().get().getTimestamp())); + savepointTime, mdtMetaClient.getCommitsTimeline().firstInstant().get().requestedTime())); deleteMDT = true; } } @@ -821,7 +823,7 @@ public HoodieRestoreMetadata restoreToInstant(final String savepointToRestoreTim private Pair> scheduleAndGetRestorePlan(final String savepointToRestoreTimestamp, HoodieTable table) throws IOException { Option failedRestore = table.getRestoreTimeline().filterInflightsAndRequested().lastInstant(); if (failedRestore.isPresent() && savepointToRestoreTimestamp.equals(RestoreUtils.getSavepointToRestoreTimestamp(table, failedRestore.get()))) { - return Pair.of(failedRestore.get().getTimestamp(), Option.of(RestoreUtils.getRestorePlan(table.getMetaClient(), failedRestore.get()))); + return Pair.of(failedRestore.get().requestedTime(), Option.of(RestoreUtils.getRestorePlan(table.getMetaClient(), failedRestore.get()))); } final String restoreInstantTimestamp = createNewInstantTime(); return Pair.of(restoreInstantTimestamp, table.scheduleRestore(context, restoreInstantTimestamp, savepointToRestoreTimestamp)); @@ -958,7 +960,7 @@ private void startCommit(String instantTime, String actionType, HoodieTableMetaC ValidationUtils.checkArgument(inflightRestoreTimeline.countInstants() == 0, "Found pending restore in active timeline. Please complete the restore fully before proceeding. As of now, " + "table could be in an inconsistent state. Pending restores: " + Arrays.toString(inflightRestoreTimeline.getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()).toArray())); + .map(HoodieInstant::requestedTime).collect(Collectors.toList()).toArray())); if (config.getFailedWritesCleanPolicy().isLazy()) { this.heartbeatClient.start(instantTime); @@ -967,7 +969,7 @@ private void startCommit(String instantTime, String actionType, HoodieTableMetaC if (ClusteringUtils.isClusteringOrReplaceCommitAction(actionType)) { metaClient.getActiveTimeline().createRequestedCommitWithReplaceMetadata(instantTime, actionType); } else { - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, actionType, + metaClient.getActiveTimeline().createNewInstant(metaClient.createNewInstant(HoodieInstant.State.REQUESTED, actionType, instantTime)); } } @@ -1021,7 +1023,7 @@ public Option index(String indexInstantTime) { public void dropIndex(List metadataPartitions) { HoodieTable table = createTable(config); String dropInstant = createNewInstantTime(); - HoodieInstant ownerInstant = new HoodieInstant(true, HoodieTimeline.INDEXING_ACTION, dropInstant); + HoodieInstant ownerInstant = table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.INDEXING_ACTION, dropInstant); this.txnManager.beginTransaction(Option.of(ownerInstant), Option.empty()); try { context.setJobStatus(this.getClass().getSimpleName(), "Dropping partitions from metadata table: " + config.getTableName()); @@ -1258,7 +1260,8 @@ public HoodieMetrics getMetrics() { protected void doInitTable(WriteOperationType operationType, HoodieTableMetaClient metaClient, Option instantTime) { Option ownerInstant = Option.empty(); if (instantTime.isPresent()) { - ownerInstant = Option.of(new HoodieInstant(true, CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime.get())); + ownerInstant = Option.of(metaClient.createNewInstant(HoodieInstant.State.INFLIGHT, CommitUtils.getCommitActionType(operationType, + metaClient.getTableType()), instantTime.get())); } executeUsingTxnManager(ownerInstant, () -> { tryUpgrade(metaClient, instantTime); @@ -1385,7 +1388,7 @@ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { || s.getAction().equals(HoodieActiveTimeline.REPLACE_COMMIT_ACTION)) .lastInstant(); if (lastInstant.isPresent()) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstant.get(), activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); String extraSchema = commitMetadata.getExtraMetadata().get(SCHEMA_KEY); if (!StringUtils.isNullOrEmpty(extraSchema)) { @@ -1436,7 +1439,7 @@ protected void tryUpgrade(HoodieTableMetaClient metaClient, Option insta UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(metaClient, config, context, upgradeDowngradeHelper); - if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) { + if (upgradeDowngrade.needsUpgradeOrDowngrade(config.getWriteVersion())) { // Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits List instantsToRollback = tableServiceClient.getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, instantTime); @@ -1595,11 +1598,11 @@ public void commitTableChange(InternalSchema newSchema, HoodieTableMetaClient me startCommitWithTime(instantTime, commitActionType, metaClient); config.setSchema(schema.toString()); HoodieActiveTimeline timeLine = metaClient.getActiveTimeline(); - HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); + HoodieInstant requested = metaClient.createNewInstant(State.REQUESTED, commitActionType, instantTime); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); metadata.setOperationType(WriteOperationType.ALTER_SCHEMA); try { - timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(metadata)); + timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata)); } catch (IOException io) { throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java index e35a855dc840..619cfbdbd82b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/CompactionAdminClient.java @@ -30,7 +30,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CompactionUtils; @@ -55,7 +55,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * Client to perform admin operations related to compaction. @@ -107,21 +108,21 @@ public List validateCompactionPlan(HoodieTableMetaClient met public List unscheduleCompactionPlan(String compactionInstant, boolean skipValidation, int parallelism, boolean dryRun) throws Exception { HoodieTableMetaClient metaClient = createMetaClient(false); - + InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); // Only if all operations are successfully executed if (!dryRun) { // Overwrite compaction request with empty compaction operations - HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant); - StoragePath inflightPath = new StoragePath(metaClient.getMetaPath(), inflight.getFileName()); + HoodieInstant inflight = metaClient.createNewInstant(State.INFLIGHT, COMPACTION_ACTION, compactionInstant); + StoragePath inflightPath = new StoragePath(metaClient.getMetaPath(), metaClient.getInstantFileNameGenerator().getFileName(inflight)); if (metaClient.getStorage().exists(inflightPath)) { // We need to rollback data-files because of this inflight compaction before unscheduling throw new IllegalStateException("Please rollback the inflight compaction before unscheduling"); } // Leave the trace in aux folder but delete from metapath. // TODO: Add a rollback instant but for compaction - HoodieInstant instant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant); + HoodieInstant instant = metaClient.createNewInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstant); boolean deleted = metaClient.getStorage().deleteFile( - new StoragePath(metaClient.getMetaPath(), instant.getFileName())); + new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(instant))); ValidationUtils.checkArgument(deleted, "Unable to delete compaction instant."); } return new ArrayList<>(); @@ -140,6 +141,7 @@ public List unscheduleCompactionPlan(String compactionInstant, b public List unscheduleCompactionFileId(HoodieFileGroupId fgId, boolean skipValidation, boolean dryRun) throws Exception { HoodieTableMetaClient metaClient = createMetaClient(false); + InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); if (!dryRun) { // Ready to remove this file-Id from compaction request @@ -156,15 +158,15 @@ public List unscheduleCompactionFileId(HoodieFileGroupId fgId, b HoodieCompactionPlan newPlan = HoodieCompactionPlan.newBuilder().setOperations(newOps).setExtraMetadata(plan.getExtraMetadata()).build(); HoodieInstant inflight = - new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()); - StoragePath inflightPath = new StoragePath(metaClient.getMetaPath(), inflight.getFileName()); + metaClient.createNewInstant(State.INFLIGHT, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()); + StoragePath inflightPath = new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(inflight)); if (metaClient.getStorage().exists(inflightPath)) { // revert if in inflight state metaClient.getActiveTimeline().revertInstantFromInflightToRequested(inflight); } // Overwrite compaction plan with updated info metaClient.getActiveTimeline().saveToCompactionRequested( - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()), + metaClient.createNewInstant(State.REQUESTED, COMPACTION_ACTION, compactionOperationWithInstant.getLeft()), TimelineMetadataUtils.serializeCompactionPlan(newPlan), true); } return new ArrayList<>(); @@ -194,7 +196,7 @@ private static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient meta throws IOException { return TimelineMetadataUtils.deserializeCompactionPlan( metaClient.getActiveTimeline().readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(compactionInstant)).get()); + metaClient.getInstantGenerator().getCompactionRequestedInstant(compactionInstant)).get()); } /** @@ -270,7 +272,7 @@ private ValidationOpResult validateCompactionOperation(HoodieTableMetaClient met + logFilesInCompactionOp + ", Got :" + logFilesInFileSlice); Set diff = logFilesInFileSlice.stream().filter(lf -> !logFilesInCompactionOp.contains(lf)) .collect(Collectors.toSet()); - ValidationUtils.checkArgument(diff.stream().allMatch(lf -> HoodieTimeline.compareTimestamps(lf.getDeltaCommitTime(), GREATER_THAN_OR_EQUALS, compactionInstant)), + ValidationUtils.checkArgument(diff.stream().allMatch(lf -> compareTimestamps(lf.getDeltaCommitTime(), GREATER_THAN_OR_EQUALS, compactionInstant)), "There are some log-files which are neither specified in compaction plan " + "nor present after compaction request instant. Some of these :" + diff); } else { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTableServiceManagerClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTableServiceManagerClient.java index 25e8e7b3c2bd..9f6fbe895158 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTableServiceManagerClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTableServiceManagerClient.java @@ -119,7 +119,7 @@ public Option executeCompaction() { String instantRange = StringUtils.join(metaClient.reloadActiveTimeline() .filterPendingCompactionTimeline() .getInstantsAsStream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .toArray(String[]::new), ","); executeRequest(EXECUTE_COMPACTION, getDefaultParams(Action.REQUEST, instantRange)); @@ -135,7 +135,7 @@ public Option executeClean() { .getCleanerTimeline() .filterInflightsAndRequested() .getInstantsAsStream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .toArray(String[]::new), ","); executeRequest(EXECUTE_CLEAN, getDefaultParams(Action.REQUEST, instantRange)); @@ -150,7 +150,7 @@ public Option executeClustering() { metaClient.reloadActiveTimeline(); String instantRange = StringUtils.join(ClusteringUtils.getPendingClusteringInstantTimes(metaClient) .stream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .toArray(String[]::new), ","); executeRequest(EXECUTE_CLUSTERING, getDefaultParams(Action.REQUEST, instantRange)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java index d8ae866b70e5..3351f753776c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java @@ -19,352 +19,22 @@ package org.apache.hudi.client.timeline; -import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.ActiveAction; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineUtils; -import org.apache.hudi.common.util.ClusteringUtils; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.CompactionUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieLockException; -import org.apache.hudi.metadata.HoodieTableMetadata; -import org.apache.hudi.metrics.HoodieMetrics; -import org.apache.hudi.table.HoodieTable; -import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; -import org.apache.hudi.table.marker.WriteMarkers; -import org.apache.hudi.table.marker.WriteMarkersFactory; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.IOException; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.function.Consumer; -import java.util.stream.Collectors; -import java.util.stream.Stream; - -import static org.apache.hudi.client.utils.ArchivalUtils.getMinAndMaxInstantsToKeep; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; /** * Archiver to bound the growth of files under .hoodie meta path. */ -public class HoodieTimelineArchiver { - - private static final Logger LOG = LoggerFactory.getLogger(HoodieTimelineArchiver.class); - - private final HoodieWriteConfig config; - private final int maxInstantsToKeep; - private final int minInstantsToKeep; - private final HoodieTable table; - private final HoodieTableMetaClient metaClient; - private final TransactionManager txnManager; - - private final LSMTimelineWriter timelineWriter; - private final HoodieMetrics metrics; +public interface HoodieTimelineArchiver { - public HoodieTimelineArchiver(HoodieWriteConfig config, HoodieTable table) { - this.config = config; - this.table = table; - this.metaClient = table.getMetaClient(); - this.txnManager = new TransactionManager(config, table.getStorage()); - this.timelineWriter = LSMTimelineWriter.getInstance(config, table); - Pair minAndMaxInstants = getMinAndMaxInstantsToKeep(table, metaClient); - this.minInstantsToKeep = minAndMaxInstants.getLeft(); - this.maxInstantsToKeep = minAndMaxInstants.getRight(); - this.metrics = new HoodieMetrics(config, metaClient.getStorage()); - } - - public int archiveIfRequired(HoodieEngineContext context) throws IOException { + default int archiveIfRequired(HoodieEngineContext context) throws IOException { return archiveIfRequired(context, false); } /** * Check if commits need to be archived. If yes, archive commits. */ - public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException { - try { - if (acquireLock) { - // there is no owner or instant time per se for archival. - txnManager.beginTransaction(Option.empty(), Option.empty()); - } - } catch (HoodieLockException e) { - LOG.error("Fail to begin transaction", e); - return 0; - } - - try { - // Sort again because the cleaning and rollback instants could break the sequence. - List instantsToArchive = getInstantsToArchive().sorted().collect(Collectors.toList()); - if (!instantsToArchive.isEmpty()) { - LOG.info("Archiving and deleting instants {}", instantsToArchive); - Consumer exceptionHandler = e -> { - if (this.config.isFailOnTimelineArchivingEnabled()) { - throw new HoodieException(e); - } - }; - this.timelineWriter.write(instantsToArchive, Option.of(action -> deleteAnyLeftOverMarkers(context, action)), Option.of(exceptionHandler)); - LOG.debug("Deleting archived instants"); - deleteArchivedInstants(instantsToArchive, context); - // triggers compaction and cleaning only after archiving action - this.timelineWriter.compactAndClean(context); - } else { - LOG.info("No Instants to archive"); - } - return instantsToArchive.size(); - } finally { - if (acquireLock) { - txnManager.endTransaction(Option.empty()); - } - } - } - - private List getCleanAndRollbackInstantsToArchive(HoodieInstant latestCommitInstantToArchive) { - HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() - .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION, HoodieTimeline.ROLLBACK_ACTION)) - .filterCompletedInstants(); - - // Since the commit instants to archive is continuous, we can use the latest commit instant to archive as the - // right boundary to collect the clean or rollback instants to archive. - // - // latestCommitInstantToArchive - // v - // | commit1 clean1 commit2 commit3 clean2 commit4 rollback1 commit5 | commit6 clean3 commit7 ... - // | <------------------ instants to archive --------------------> | - // - // CommitInstantsToArchive: commit1, commit2, commit3, commit4, commit5 - // CleanAndRollbackInstantsToArchive: clean1, clean2, rollback1 - - return cleanAndRollbackTimeline.getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN, latestCommitInstantToArchive.getTimestamp())) - .collect(Collectors.toList()); - } - - private List getCommitInstantsToArchive() throws IOException { - HoodieTimeline completedCommitsTimeline = table.getCompletedCommitsTimeline(); - - if (completedCommitsTimeline.countInstants() <= maxInstantsToKeep) { - return Collections.emptyList(); - } - - // Step1: Get all candidates of earliestInstantToRetain. - List> earliestInstantToRetainCandidates = new ArrayList<>(); - - // 1. Earliest commit to retain is the greatest completed commit, that is less than the earliest pending instant. - // In some cases when inflight is the lowest commit then earliest commit to retain will be equal to the earliest - // inflight commit. - Option earliestPendingInstant = table.getActiveTimeline() - .getWriteTimeline() - .filter(instant -> !instant.isCompleted()) - .firstInstant(); - - Option earliestCommitToRetain; - if (earliestPendingInstant.isPresent()) { - Option completedCommitBeforeEarliestPendingInstant = Option.fromJavaOptional(completedCommitsTimeline - .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPendingInstant.get().getTimestamp())) - .getReverseOrderedInstants().findFirst()); - // Check if the completed instant is higher than the earliest inflight instant - // in that case update the earliestCommitToRetain to earliestInflight commit time. - if (!completedCommitBeforeEarliestPendingInstant.isPresent()) { - earliestCommitToRetain = earliestPendingInstant; - } else { - earliestCommitToRetain = completedCommitBeforeEarliestPendingInstant; - } - } else { - earliestCommitToRetain = Option.empty(); - } - earliestInstantToRetainCandidates.add(earliestCommitToRetain); - - // 2. For Merge-On-Read table, inline or async compaction is enabled - // We need to make sure that there are enough delta commits in the active timeline - // to trigger compaction scheduling, when the trigger strategy of compaction is - // NUM_COMMITS or NUM_AND_TIME. - Option earliestInstantToRetainForCompaction = - (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ - && (config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_COMMITS - || config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_AND_TIME)) - ? CompactionUtils.getEarliestInstantToRetainForCompaction( - table.getActiveTimeline(), config.getInlineCompactDeltaCommitMax()) - : Option.empty(); - earliestInstantToRetainCandidates.add(earliestInstantToRetainForCompaction); - - // 3. The clustering commit instant can not be archived unless we ensure that the replaced files have been cleaned, - // without the replaced files metadata on the timeline, the fs view would expose duplicates for readers. - // Meanwhile, when inline or async clustering is enabled, we need to ensure that there is a commit in the active timeline - // to check whether the file slice generated in pending clustering after archive isn't committed. - Option earliestInstantToRetainForClustering = - ClusteringUtils.getEarliestInstantToRetainForClustering(table.getActiveTimeline(), table.getMetaClient(), - config.getCleanerPolicy()); - earliestInstantToRetainCandidates.add(earliestInstantToRetainForClustering); - - // 4. If metadata table is enabled, do not archive instants which are more recent than the last compaction on the - // metadata table. - if (table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { - try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create( - table.getContext(), table.getStorage(), config.getMetadataConfig(), config.getBasePath())) { - Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); - if (!latestCompactionTime.isPresent()) { - LOG.info("Not archiving as there is no compaction yet on the metadata table"); - return Collections.emptyList(); - } else { - LOG.info("Limiting archiving of instants to latest compaction on metadata table at {}", latestCompactionTime.get()); - earliestInstantToRetainCandidates.add( - completedCommitsTimeline.findInstantsModifiedAfterByCompletionTime(latestCompactionTime.get()).firstInstant()); - } - } catch (Exception e) { - throw new HoodieException("Error limiting instant archival based on metadata table", e); - } - } - - // 5. If this is a metadata table, do not archive the commits that live in data set - // active timeline. This is required by metadata table, - // see HoodieTableMetadataUtil#processRollbackMetadata for details. - if (table.isMetadataTable()) { - HoodieTableMetaClient dataMetaClient = HoodieTableMetaClient.builder() - .setBasePath(HoodieTableMetadata.getDatasetBasePath(config.getBasePath())) - .setConf(metaClient.getStorageConf().newInstance()) - .build(); - Option qualifiedEarliestInstant = - TimelineUtils.getEarliestInstantForMetadataArchival( - dataMetaClient.getActiveTimeline(), config.shouldArchiveBeyondSavepoint()); - - // Do not archive the instants after the earliest commit (COMMIT, DELTA_COMMIT, and - // REPLACE_COMMIT only, considering non-savepoint commit only if enabling archive - // beyond savepoint) and the earliest inflight instant (all actions). - // This is required by metadata table, see HoodieTableMetadataUtil#processRollbackMetadata - // for details. - // Note that we cannot blindly use the earliest instant of all actions, because CLEAN and - // ROLLBACK instants are archived separately apart from commits (check - // HoodieTimelineArchiver#getCleanInstantsToArchive). If we do so, a very old completed - // CLEAN or ROLLBACK instant can block the archive of metadata table timeline and causes - // the active timeline of metadata table to be extremely long, leading to performance issues - // for loading the timeline. - earliestInstantToRetainCandidates.add(qualifiedEarliestInstant); - } - - // Choose the instant in earliestInstantToRetainCandidates with the smallest - // timestamp as earliestInstantToRetain. - java.util.Optional earliestInstantToRetain = earliestInstantToRetainCandidates - .stream() - .filter(Option::isPresent) - .map(Option::get) - .min(HoodieInstant.INSTANT_TIME_COMPARATOR); - - // Step2: We cannot archive any commits which are made after the first savepoint present, - // unless HoodieArchivalConfig#ARCHIVE_BEYOND_SAVEPOINT is enabled. - Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); - Set savepointTimestamps = table.getSavepointTimestamps(); - - Stream instantToArchiveStream = completedCommitsTimeline.getInstantsAsStream() - .filter(s -> { - if (config.shouldArchiveBeyondSavepoint()) { - // skip savepoint commits and proceed further - return !savepointTimestamps.contains(s.getTimestamp()); - } else { - // if no savepoint present, then don't filter - // stop at first savepoint commit - return !firstSavepoint.isPresent() || compareTimestamps(s.getTimestamp(), LESSER_THAN, firstSavepoint.get().getTimestamp()); - } - }).filter(s -> earliestInstantToRetain - .map(instant -> compareTimestamps(s.getTimestamp(), LESSER_THAN, instant.getTimestamp())) - .orElse(true)); - return instantToArchiveStream.limit(completedCommitsTimeline.countInstants() - minInstantsToKeep) - .collect(Collectors.toList()); - } - - private Stream getInstantsToArchive() throws IOException { - if (config.isMetaserverEnabled()) { - return Stream.empty(); - } - - // First get commit instants to archive. - List instantsToArchive = getCommitInstantsToArchive(); - if (!instantsToArchive.isEmpty()) { - HoodieInstant latestCommitInstantToArchive = instantsToArchive.get(instantsToArchive.size() - 1); - // Then get clean and rollback instants to archive. - List cleanAndRollbackInstantsToArchive = - getCleanAndRollbackInstantsToArchive(latestCommitInstantToArchive); - instantsToArchive.addAll(cleanAndRollbackInstantsToArchive); - instantsToArchive.sort(HoodieInstant.INSTANT_TIME_COMPARATOR); - } - - // For archive, we need to include instant's all states. - // The re-instantiation of the timeline may result in inconsistencies with the existing meta client active timeline, - // When there is no lock guard of the archiving process, the 'raw' timeline could contain less distinct instants - // because of the metadata file removing from another archiving process. - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - Map, List> groupByTsAction = rawActiveTimeline.getInstantsAsStream() - .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), - HoodieInstant.getComparableAction(i.getAction())))); - - return instantsToArchive.stream().flatMap(hoodieInstant -> { - List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), - HoodieInstant.getComparableAction(hoodieInstant.getAction()))); - if (instantsToStream != null) { - return Stream.of(ActiveAction.fromInstants(instantsToStream)); - } else { - // if a concurrent writer archived the instant - return Stream.empty(); - } - }); - } - - private boolean deleteArchivedInstants(List activeActions, HoodieEngineContext context) { - List pendingInstants = new ArrayList<>(); - List completedInstants = new ArrayList<>(); - - for (ActiveAction activeAction : activeActions) { - completedInstants.addAll(activeAction.getCompletedInstants()); - pendingInstants.addAll(activeAction.getPendingInstants()); - } - - context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); - // Delete the metadata files - // in HoodieInstant.State sequence: requested -> inflight -> completed, - // this is important because when a COMPLETED metadata file is removed first, - // other monitors on the timeline(such as the compaction or clustering services) would - // mistakenly recognize the pending file as a pending operation, - // then all kinds of weird bugs occur. - HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); - if (!pendingInstants.isEmpty()) { - context.foreach( - pendingInstants, - instant -> activeTimeline.deleteInstantFileIfExists(instant), - Math.min(pendingInstants.size(), config.getArchiveDeleteParallelism()) - ); - } - if (!completedInstants.isEmpty()) { - // Due to the concurrency between deleting completed instants and reading data, - // there may be hole in the timeline, which can lead to errors when reading data. - // Therefore, the concurrency of deleting completed instants is temporarily disabled, - // and instants are deleted in ascending order to prevent the occurrence of such holes. - // See HUDI-7207 and #10325. - completedInstants.stream() - .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); - } - - return true; - } - - private void deleteAnyLeftOverMarkers(HoodieEngineContext context, ActiveAction activeAction) { - WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, activeAction.getInstantTime()); - if (writeMarkers.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) { - LOG.info("Cleaned up left over marker directory for instant: {}", activeAction); - } - } + public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/TimelineArchivers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/TimelineArchivers.java new file mode 100644 index 000000000000..315a6db481d6 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/TimelineArchivers.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.timeline; + +import org.apache.hudi.client.timeline.versioning.v1.TimelineArchiverV1; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.table.HoodieTable; + +public class TimelineArchivers { + + public static HoodieTimelineArchiver getInstance(TimelineLayoutVersion layoutVersion, + HoodieWriteConfig config, + HoodieTable table) { + if (layoutVersion.equals(TimelineLayoutVersion.LAYOUT_VERSION_0) || layoutVersion.equals(TimelineLayoutVersion.LAYOUT_VERSION_1)) { + return new TimelineArchiverV1<>(config, table); + } else if (layoutVersion.equals(TimelineLayoutVersion.LAYOUT_VERSION_2)) { + return new TimelineArchiverV2<>(config, table); + } else { + throw new HoodieException("Unknown table layout version : " + layoutVersion.getVersion()); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java new file mode 100644 index 000000000000..78fd42bbe610 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v1/TimelineArchiverV1.java @@ -0,0 +1,443 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.timeline.versioning.v1; + +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieArchivedLogFile; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.MetadataConversionUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.versioning.v1.ActiveTimelineV1; +import org.apache.hudi.common.table.timeline.versioning.v1.ArchivedTimelineV1; +import org.apache.hudi.common.table.timeline.versioning.v1.InstantComparatorV1; +import org.apache.hudi.common.table.timeline.versioning.v1.InstantFileNameGeneratorV1; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; +import org.apache.hudi.table.marker.WriteMarkers; +import org.apache.hudi.table.marker.WriteMarkersFactory; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.client.utils.ArchivalUtils.getMinAndMaxInstantsToKeep; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + +/** + * Archiver to bound the growth of files under .hoodie meta path. + */ +public class TimelineArchiverV1 implements HoodieTimelineArchiver { + + private static final Logger LOG = LoggerFactory.getLogger(TimelineArchiverV1.class); + + private final StoragePath archiveFilePath; + private final HoodieWriteConfig config; + private Writer writer; + private final int maxInstantsToKeep; + private final int minInstantsToKeep; + private final HoodieTable table; + private final HoodieTableMetaClient metaClient; + private final TransactionManager txnManager; + + public TimelineArchiverV1(HoodieWriteConfig config, HoodieTable table) { + this.config = config; + this.table = table; + this.metaClient = table.getMetaClient(); + this.archiveFilePath = ArchivedTimelineV1.getArchiveLogPath(metaClient.getArchivePath()); + this.txnManager = new TransactionManager(config, table.getMetaClient().getStorage()); + Pair minAndMaxInstants = getMinAndMaxInstantsToKeep(table, metaClient); + this.minInstantsToKeep = minAndMaxInstants.getLeft(); + this.maxInstantsToKeep = minAndMaxInstants.getRight(); + } + + private Writer openWriter() { + try { + if (this.writer == null) { + return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent()).withDeltaCommit("") + .withFileId(archiveFilePath.getName()).withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) + .withStorage(metaClient.getStorage()).build(); + } else { + return this.writer; + } + } catch (IOException e) { + throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); + } + } + + public Writer reOpenWriter() { + try { + if (this.writer != null) { + this.writer.close(); + this.writer = null; + } + this.writer = openWriter(); + return writer; + } catch (IOException e) { + throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); + } + } + + private void close() { + try { + if (this.writer != null) { + this.writer.close(); + } + } catch (IOException e) { + throw new HoodieException("Unable to close HoodieLogFormat writer", e); + } + } + + @Override + public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException { + //NOTE: We permanently disable merging archive files. This is different from 0.15 behavior. + try { + if (acquireLock) { + // there is no owner or instant time per se for archival. + txnManager.beginTransaction(Option.empty(), Option.empty()); + } + List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); + boolean success = true; + if (!instantsToArchive.isEmpty()) { + this.writer = openWriter(); + LOG.info("Archiving instants " + instantsToArchive); + archive(context, instantsToArchive); + LOG.info("Deleting archived instants " + instantsToArchive); + success = deleteArchivedInstants(instantsToArchive, context); + } else { + LOG.info("No Instants to archive"); + } + + return instantsToArchive.size(); + } finally { + close(); + if (acquireLock) { + txnManager.endTransaction(Option.empty()); + } + } + } + + private Stream getCleanInstantsToArchive() { + HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION, HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants(); + return cleanAndRollbackTimeline.getInstantsAsStream() + .collect(Collectors.groupingBy(HoodieInstant::getAction)).values().stream() + .map(hoodieInstants -> { + if (hoodieInstants.size() > this.maxInstantsToKeep) { + return hoodieInstants.subList(0, hoodieInstants.size() - this.minInstantsToKeep); + } else { + return new ArrayList(); + } + }).flatMap(Collection::stream); + } + + private Stream getCommitInstantsToArchive() throws IOException { + // TODO (na) : Add a way to return actions associated with a timeline and then merge/unify + // with logic above to avoid Stream.concat + HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); + + // Get the oldest inflight instant and a completed commit before this inflight instant. + Option oldestPendingInstant = table.getActiveTimeline() + .getWriteTimeline() + .filter(instant -> !instant.isCompleted()) + .firstInstant(); + + // Oldest commit to retain is the greatest completed commit, that is less than the oldest pending instant. + // In some cases when inflight is the lowest commit then oldest commit to retain will be equal to oldest + // inflight commit. + Option oldestCommitToRetain; + if (oldestPendingInstant.isPresent()) { + Option completedCommitBeforeOldestPendingInstant = + Option.fromJavaOptional(commitTimeline.getReverseOrderedInstants() + .filter(instant -> compareTimestamps(instant.requestedTime(), + LESSER_THAN, oldestPendingInstant.get().requestedTime())).findFirst()); + // Check if the completed instant is higher than the oldest inflight instant + // in that case update the oldestCommitToRetain to oldestInflight commit time. + if (!completedCommitBeforeOldestPendingInstant.isPresent() + || compareTimestamps(oldestPendingInstant.get().requestedTime(), + LESSER_THAN, completedCommitBeforeOldestPendingInstant.get().requestedTime())) { + oldestCommitToRetain = oldestPendingInstant; + } else { + oldestCommitToRetain = completedCommitBeforeOldestPendingInstant; + } + } else { + oldestCommitToRetain = Option.empty(); + } + + // NOTE: We cannot have any holes in the commit timeline. + // We cannot archive any commits which are made after the first savepoint present, + // unless HoodieArchivalConfig#ARCHIVE_BEYOND_SAVEPOINT is enabled. + Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); + Set savepointTimestamps = table.getSavepointTimestamps(); + if (!commitTimeline.empty() && commitTimeline.countInstants() > maxInstantsToKeep) { + // For Merge-On-Read table, inline or async compaction is enabled + // We need to make sure that there are enough delta commits in the active timeline + // to trigger compaction scheduling, when the trigger strategy of compaction is + // NUM_COMMITS or NUM_AND_TIME. + Option oldestInstantToRetainForCompaction = + (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ + && (config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_COMMITS + || config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_AND_TIME)) + ? CompactionUtils.getEarliestInstantToRetainForCompaction( + table.getActiveTimeline(), config.getInlineCompactDeltaCommitMax()) + : Option.empty(); + + // The clustering commit instant can not be archived unless we ensure that the replaced files have been cleaned, + // without the replaced files metadata on the timeline, the fs view would expose duplicates for readers. + // Meanwhile, when inline or async clustering is enabled, we need to ensure that there is a commit in the active timeline + // to check whether the file slice generated in pending clustering after archive isn't committed. + Option oldestInstantToRetainForClustering = + ClusteringUtils.getEarliestInstantToRetainForClustering(table.getActiveTimeline(), table.getMetaClient(), config.getCleanerPolicy()); + + // Actually do the commits + Stream instantToArchiveStream = commitTimeline.getInstantsAsStream() + .filter(s -> { + if (config.shouldArchiveBeyondSavepoint()) { + // skip savepoint commits and proceed further + return !savepointTimestamps.contains(s.requestedTime()); + } else { + // if no savepoint present, then don't filter + // stop at first savepoint commit + return !(firstSavepoint.isPresent() && compareTimestamps(firstSavepoint.get().requestedTime(), LESSER_THAN_OR_EQUALS, s.requestedTime())); + } + }).filter(s -> { + // oldestCommitToRetain is the highest completed commit instant that is less than the oldest inflight instant. + // By filtering out any commit >= oldestCommitToRetain, we can ensure there are no gaps in the timeline + // when inflight commits are present. + return oldestCommitToRetain + .map(instant -> compareTimestamps(instant.requestedTime(), GREATER_THAN, s.requestedTime())) + .orElse(true); + }).filter(s -> + oldestInstantToRetainForCompaction.map(instantToRetain -> + compareTimestamps(s.requestedTime(), LESSER_THAN, instantToRetain.requestedTime())) + .orElse(true) + ).filter(s -> + oldestInstantToRetainForClustering.map(instantToRetain -> + compareTimestamps(s.requestedTime(), LESSER_THAN, instantToRetain.requestedTime())) + .orElse(true) + ); + return instantToArchiveStream.limit(commitTimeline.countInstants() - minInstantsToKeep); + } else { + return Stream.empty(); + } + } + + private Stream getInstantsToArchive() throws IOException { + Stream instants = Stream.concat(getCleanInstantsToArchive(), getCommitInstantsToArchive()); + if (config.isMetaserverEnabled()) { + return Stream.empty(); + } + + // For archiving and cleaning instants, we need to include intermediate state files if they exist + HoodieActiveTimeline rawActiveTimeline = new ActiveTimelineV1(metaClient, false); + Map, List> groupByTsAction = rawActiveTimeline.getInstantsAsStream() + .collect(Collectors.groupingBy(i -> Pair.of(i.requestedTime(), + InstantComparatorV1.getComparableAction(i.getAction())))); + + // If metadata table is enabled, do not archive instants which are more recent than the last compaction on the + // metadata table. + if (table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create(table.getContext(), table.getStorage(), config.getMetadataConfig(), config.getBasePath())) { + Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); + if (!latestCompactionTime.isPresent()) { + LOG.info("Not archiving as there is no compaction yet on the metadata table"); + instants = Stream.empty(); + } else { + LOG.info("Limiting archiving of instants to latest compaction on metadata table at " + latestCompactionTime.get()); + instants = instants.filter(instant -> compareTimestamps(instant.requestedTime(), LESSER_THAN, + latestCompactionTime.get())); + } + } catch (Exception e) { + throw new HoodieException("Error limiting instant archival based on metadata table", e); + } + } + + if (table.isMetadataTable()) { + HoodieTableMetaClient dataMetaClient = HoodieTableMetaClient.builder() + .setBasePath(HoodieTableMetadata.getDatasetBasePath(config.getBasePath())) + .setConf(metaClient.getStorageConf()) + .build(); + Option qualifiedEarliestInstant = + TimelineUtils.getEarliestInstantForMetadataArchival( + dataMetaClient.getActiveTimeline(), config.shouldArchiveBeyondSavepoint()); + + // Do not archive the instants after the earliest commit (COMMIT, DELTA_COMMIT, and + // REPLACE_COMMIT only, considering non-savepoint commit only if enabling archive + // beyond savepoint) and the earliest inflight instant (all actions). + // This is required by metadata table, see HoodieTableMetadataUtil#processRollbackMetadata + // for details. + // Note that we cannot blindly use the earliest instant of all actions, because CLEAN and + // ROLLBACK instants are archived separately apart from commits (check + // TimelineArchiverV1#getCleanInstantsToArchive). If we do so, a very old completed + // CLEAN or ROLLBACK instant can block the archive of metadata table timeline and causes + // the active timeline of metadata table to be extremely long, leading to performance issues + // for loading the timeline. + if (qualifiedEarliestInstant.isPresent()) { + instants = instants.filter(instant -> + compareTimestamps( + instant.requestedTime(), + LESSER_THAN, + qualifiedEarliestInstant.get().requestedTime())); + } + } + + return instants.flatMap(hoodieInstant -> { + List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.requestedTime(), + InstantComparatorV1.getComparableAction(hoodieInstant.getAction()))); + if (instantsToStream != null) { + return instantsToStream.stream(); + } else { + // if a concurrent writer archived the instant + return Stream.empty(); + } + }); + } + + private boolean deleteArchivedInstants(List archivedInstants, HoodieEngineContext context) throws IOException { + LOG.info("Deleting instants " + archivedInstants); + + List pendingInstants = new ArrayList<>(); + List completedInstants = new ArrayList<>(); + + for (HoodieInstant instant : archivedInstants) { + if (instant.isCompleted()) { + completedInstants.add(instant); + } else { + pendingInstants.add(instant); + } + } + + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); + // Delete the metadata files + // in HoodieInstant.State sequence: requested -> inflight -> completed, + // this is important because when a COMPLETED metadata file is removed first, + // other monitors on the timeline(such as the compaction or clustering services) would + // mistakenly recognize the pending file as a pending operation, + // then all kinds of weird bugs occur. + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + if (!pendingInstants.isEmpty()) { + context.foreach( + pendingInstants, + instant -> activeTimeline.deleteInstantFileIfExists(instant), + Math.min(pendingInstants.size(), config.getArchiveDeleteParallelism()) + ); + } + if (!completedInstants.isEmpty()) { + // Due to the concurrency between deleting completed instants and reading data, + // there may be hole in the timeline, which can lead to errors when reading data. + // Therefore, the concurrency of deleting completed instants is temporarily disabled, + // and instants are deleted in ascending order to prevent the occurrence of such holes. + // See HUDI-7207 and #10325. + completedInstants.stream() + .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); + } + + return true; + } + + public void archive(HoodieEngineContext context, List instants) throws HoodieCommitException { + try { + Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); + LOG.info("Wrapper schema " + wrapperSchema.toString()); + List records = new ArrayList<>(); + for (HoodieInstant hoodieInstant : instants) { + try { + deleteAnyLeftOverMarkers(context, hoodieInstant); + records.add(convertToAvroRecord(hoodieInstant)); + if (records.size() >= this.config.getCommitArchivalBatchSize()) { + writeToFile(wrapperSchema, records); + } + } catch (Exception e) { + InstantFileNameGenerator fileNameFactory = new InstantFileNameGeneratorV1(); + LOG.error("Failed to archive commits, .commit file: " + fileNameFactory.getFileName(hoodieInstant), e); + if (this.config.isFailOnTimelineArchivingEnabled()) { + throw e; + } + } + } + writeToFile(wrapperSchema, records); + } catch (Exception e) { + throw new HoodieCommitException("Failed to archive commits", e); + } + } + + private void deleteAnyLeftOverMarkers(HoodieEngineContext context, HoodieInstant instant) { + WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instant.requestedTime()); + if (writeMarkers.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) { + LOG.info("Cleaned up left over marker directory for instant :" + instant); + } + } + + private void writeToFile(Schema wrapperSchema, List records) throws Exception { + if (records.size() > 0) { + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); + final String keyField = table.getMetaClient().getTableConfig().getRecordKeyFieldProp(); + List indexRecords = records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(indexRecords, false, header, keyField); + writer.appendBlock(block); + records.clear(); + } + } + + private IndexedRecord convertToAvroRecord(HoodieInstant hoodieInstant) + throws IOException { + return MetadataConversionUtils.createMetaWrapper(hoodieInstant, metaClient); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/LSMTimelineWriter.java similarity index 99% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/LSMTimelineWriter.java index 4986c22fd050..25113007cc6d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/LSMTimelineWriter.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.hudi.client.timeline; +package org.apache.hudi.client.timeline.versioning.v2; import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; import org.apache.hudi.common.engine.HoodieEngineContext; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java new file mode 100644 index 000000000000..21da2233358d --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/versioning/v2/TimelineArchiverV2.java @@ -0,0 +1,366 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.timeline.versioning.v2; + +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.transaction.TransactionManager; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ActiveAction; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.CompactionUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.exception.HoodieLockException; +import org.apache.hudi.metadata.HoodieTableMetadata; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; +import org.apache.hudi.table.marker.WriteMarkers; +import org.apache.hudi.table.marker.WriteMarkersFactory; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.apache.hudi.client.utils.ArchivalUtils.getMinAndMaxInstantsToKeep; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + +/** + * Archiver to bound the growth of files under .hoodie meta path. + */ +public class TimelineArchiverV2 implements HoodieTimelineArchiver { + + private static final Logger LOG = LoggerFactory.getLogger(TimelineArchiverV2.class); + + private final HoodieWriteConfig config; + private final int maxInstantsToKeep; + private final int minInstantsToKeep; + private final HoodieTable table; + private final HoodieTableMetaClient metaClient; + private final TransactionManager txnManager; + + private final LSMTimelineWriter timelineWriter; + + public TimelineArchiverV2(HoodieWriteConfig config, HoodieTable table) { + this.config = config; + this.table = table; + this.metaClient = table.getMetaClient(); + this.txnManager = new TransactionManager(config, table.getStorage()); + this.timelineWriter = LSMTimelineWriter.getInstance(config, table); + Pair minAndMaxInstants = getMinAndMaxInstantsToKeep(table, metaClient); + this.minInstantsToKeep = minAndMaxInstants.getLeft(); + this.maxInstantsToKeep = minAndMaxInstants.getRight(); + } + + @Override + public int archiveIfRequired(HoodieEngineContext context, boolean acquireLock) throws IOException { + try { + if (acquireLock) { + // there is no owner or instant time per se for archival. + txnManager.beginTransaction(Option.empty(), Option.empty()); + } + } catch (HoodieLockException e) { + LOG.error("Fail to begin transaction", e); + return 0; + } + + try { + // Sort again because the cleaning and rollback instants could break the sequence. + List instantsToArchive = getInstantsToArchive().sorted().collect(Collectors.toList()); + if (!instantsToArchive.isEmpty()) { + LOG.info("Archiving and deleting instants {}", instantsToArchive); + Consumer exceptionHandler = e -> { + if (this.config.isFailOnTimelineArchivingEnabled()) { + throw new HoodieException(e); + } + }; + this.timelineWriter.write(instantsToArchive, Option.of(action -> deleteAnyLeftOverMarkers(context, action)), Option.of(exceptionHandler)); + LOG.debug("Deleting archived instants"); + deleteArchivedInstants(instantsToArchive, context); + // triggers compaction and cleaning only after archiving action + this.timelineWriter.compactAndClean(context); + } else { + LOG.info("No Instants to archive"); + } + return instantsToArchive.size(); + } finally { + if (acquireLock) { + txnManager.endTransaction(Option.empty()); + } + } + } + + private List getCleanAndRollbackInstantsToArchive(HoodieInstant latestCommitInstantToArchive) { + HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() + .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION, HoodieTimeline.ROLLBACK_ACTION)) + .filterCompletedInstants(); + + // Since the commit instants to archive is continuous, we can use the latest commit instant to archive as the + // right boundary to collect the clean or rollback instants to archive. + // + // latestCommitInstantToArchive + // v + // | commit1 clean1 commit2 commit3 clean2 commit4 rollback1 commit5 | commit6 clean3 commit7 ... + // | <------------------ instants to archive --------------------> | + // + // CommitInstantsToArchive: commit1, commit2, commit3, commit4, commit5 + // CleanAndRollbackInstantsToArchive: clean1, clean2, rollback1 + + return cleanAndRollbackTimeline.getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), LESSER_THAN, latestCommitInstantToArchive.requestedTime())) + .collect(Collectors.toList()); + } + + private List getCommitInstantsToArchive() throws IOException { + HoodieTimeline completedCommitsTimeline = table.getCompletedCommitsTimeline(); + + if (completedCommitsTimeline.countInstants() <= maxInstantsToKeep) { + return Collections.emptyList(); + } + + // Step1: Get all candidates of earliestInstantToRetain. + List> earliestInstantToRetainCandidates = new ArrayList<>(); + + // 1. Earliest commit to retain is the greatest completed commit, that is less than the earliest pending instant. + // In some cases when inflight is the lowest commit then earliest commit to retain will be equal to the earliest + // inflight commit. + Option earliestPendingInstant = table.getActiveTimeline() + .getWriteTimeline() + .filter(instant -> !instant.isCompleted()) + .firstInstant(); + + Option earliestCommitToRetain; + if (earliestPendingInstant.isPresent()) { + Option completedCommitBeforeEarliestPendingInstant = Option.fromJavaOptional(completedCommitsTimeline + .filter(instant -> compareTimestamps(instant.requestedTime(), LESSER_THAN, earliestPendingInstant.get().requestedTime())) + .getReverseOrderedInstants().findFirst()); + // Check if the completed instant is higher than the earliest inflight instant + // in that case update the earliestCommitToRetain to earliestInflight commit time. + if (!completedCommitBeforeEarliestPendingInstant.isPresent()) { + earliestCommitToRetain = earliestPendingInstant; + } else { + earliestCommitToRetain = completedCommitBeforeEarliestPendingInstant; + } + } else { + earliestCommitToRetain = Option.empty(); + } + earliestInstantToRetainCandidates.add(earliestCommitToRetain); + + // 2. For Merge-On-Read table, inline or async compaction is enabled + // We need to make sure that there are enough delta commits in the active timeline + // to trigger compaction scheduling, when the trigger strategy of compaction is + // NUM_COMMITS or NUM_AND_TIME. + Option earliestInstantToRetainForCompaction = + (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ + && (config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_COMMITS + || config.getInlineCompactTriggerStrategy() == CompactionTriggerStrategy.NUM_AND_TIME)) + ? CompactionUtils.getEarliestInstantToRetainForCompaction( + table.getActiveTimeline(), config.getInlineCompactDeltaCommitMax()) + : Option.empty(); + earliestInstantToRetainCandidates.add(earliestInstantToRetainForCompaction); + + // 3. The clustering commit instant can not be archived unless we ensure that the replaced files have been cleaned, + // without the replaced files metadata on the timeline, the fs view would expose duplicates for readers. + // Meanwhile, when inline or async clustering is enabled, we need to ensure that there is a commit in the active timeline + // to check whether the file slice generated in pending clustering after archive isn't committed. + Option earliestInstantToRetainForClustering = + ClusteringUtils.getEarliestInstantToRetainForClustering(table.getActiveTimeline(), table.getMetaClient(), + config.getCleanerPolicy()); + earliestInstantToRetainCandidates.add(earliestInstantToRetainForClustering); + + // 4. If metadata table is enabled, do not archive instants which are more recent than the last compaction on the + // metadata table. + if (table.getMetaClient().getTableConfig().isMetadataTableAvailable()) { + try (HoodieTableMetadata tableMetadata = HoodieTableMetadata.create( + table.getContext(), table.getStorage(), config.getMetadataConfig(), config.getBasePath())) { + Option latestCompactionTime = tableMetadata.getLatestCompactionTime(); + if (!latestCompactionTime.isPresent()) { + LOG.info("Not archiving as there is no compaction yet on the metadata table"); + return Collections.emptyList(); + } else { + LOG.info("Limiting archiving of instants to latest compaction on metadata table at {}", latestCompactionTime.get()); + earliestInstantToRetainCandidates.add( + completedCommitsTimeline.findInstantsModifiedAfterByCompletionTime(latestCompactionTime.get()).firstInstant()); + } + } catch (Exception e) { + throw new HoodieException("Error limiting instant archival based on metadata table", e); + } + } + + // 5. If this is a metadata table, do not archive the commits that live in data set + // active timeline. This is required by metadata table, + // see HoodieTableMetadataUtil#processRollbackMetadata for details. + if (table.isMetadataTable()) { + HoodieTableMetaClient dataMetaClient = HoodieTableMetaClient.builder() + .setBasePath(HoodieTableMetadata.getDatasetBasePath(config.getBasePath())) + .setConf(metaClient.getStorageConf().newInstance()) + .build(); + Option qualifiedEarliestInstant = + TimelineUtils.getEarliestInstantForMetadataArchival( + dataMetaClient.getActiveTimeline(), config.shouldArchiveBeyondSavepoint()); + + // Do not archive the instants after the earliest commit (COMMIT, DELTA_COMMIT, and + // REPLACE_COMMIT only, considering non-savepoint commit only if enabling archive + // beyond savepoint) and the earliest inflight instant (all actions). + // This is required by metadata table, see HoodieTableMetadataUtil#processRollbackMetadata + // for details. + // Note that we cannot blindly use the earliest instant of all actions, because CLEAN and + // ROLLBACK instants are archived separately apart from commits (check + // TimelineArchiverV2#getCleanInstantsToArchive). If we do so, a very old completed + // CLEAN or ROLLBACK instant can block the archive of metadata table timeline and causes + // the active timeline of metadata table to be extremely long, leading to performance issues + // for loading the timeline. + earliestInstantToRetainCandidates.add(qualifiedEarliestInstant); + } + + // Choose the instant in earliestInstantToRetainCandidates with the smallest + // timestamp as earliestInstantToRetain. + //TODO: HARDCODED TIMELINE OBJECT + java.util.Optional earliestInstantToRetain = earliestInstantToRetainCandidates + .stream() + .filter(Option::isPresent) + .map(Option::get) + .min(InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + + // Step2: We cannot archive any commits which are made after the first savepoint present, + // unless HoodieArchivalConfig#ARCHIVE_BEYOND_SAVEPOINT is enabled. + Option firstSavepoint = table.getCompletedSavepointTimeline().firstInstant(); + Set savepointTimestamps = table.getSavepointTimestamps(); + + Stream instantToArchiveStream = completedCommitsTimeline.getInstantsAsStream() + .filter(s -> { + if (config.shouldArchiveBeyondSavepoint()) { + // skip savepoint commits and proceed further + return !savepointTimestamps.contains(s.requestedTime()); + } else { + // if no savepoint present, then don't filter + // stop at first savepoint commit + return !firstSavepoint.isPresent() || compareTimestamps(s.requestedTime(), LESSER_THAN, firstSavepoint.get().requestedTime()); + } + }).filter(s -> earliestInstantToRetain + .map(instant -> compareTimestamps(s.requestedTime(), LESSER_THAN, instant.requestedTime())) + .orElse(true)); + return instantToArchiveStream.limit(completedCommitsTimeline.countInstants() - minInstantsToKeep) + .collect(Collectors.toList()); + } + + private Stream getInstantsToArchive() throws IOException { + if (config.isMetaserverEnabled()) { + return Stream.empty(); + } + + // First get commit instants to archive. + List instantsToArchive = getCommitInstantsToArchive(); + if (!instantsToArchive.isEmpty()) { + HoodieInstant latestCommitInstantToArchive = instantsToArchive.get(instantsToArchive.size() - 1); + // Then get clean and rollback instants to archive. + List cleanAndRollbackInstantsToArchive = + getCleanAndRollbackInstantsToArchive(latestCommitInstantToArchive); + instantsToArchive.addAll(cleanAndRollbackInstantsToArchive); + instantsToArchive.sort(InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + // For archive, we need to include instant's all states. + // The re-instantiation of the timeline may result in inconsistencies with the existing meta client active timeline, + // When there is no lock guard of the archiving process, the 'raw' timeline could contain less distinct instants + // because of the metadata file removing from another archiving process. + //TODO: HARDCODED TIMELINE OBJECT + HoodieActiveTimeline rawActiveTimeline = new ActiveTimelineV2(metaClient, false); + Map, List> groupByTsAction = rawActiveTimeline.getInstantsAsStream() + .collect(Collectors.groupingBy(i -> Pair.of(i.requestedTime(), + InstantComparatorV2.getComparableAction(i.getAction())))); + + return instantsToArchive.stream().flatMap(hoodieInstant -> { + List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.requestedTime(), + InstantComparatorV2.getComparableAction(hoodieInstant.getAction()))); + if (instantsToStream != null) { + return Stream.of(ActiveAction.fromInstants(instantsToStream)); + } else { + // if a concurrent writer archived the instant + return Stream.empty(); + } + }); + } + + private boolean deleteArchivedInstants(List activeActions, HoodieEngineContext context) { + List pendingInstants = new ArrayList<>(); + List completedInstants = new ArrayList<>(); + + for (ActiveAction activeAction : activeActions) { + completedInstants.addAll(activeAction.getCompletedInstants()); + pendingInstants.addAll(activeAction.getPendingInstants()); + } + + context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); + // Delete the metadata files + // in HoodieInstant.State sequence: requested -> inflight -> completed, + // this is important because when a COMPLETED metadata file is removed first, + // other monitors on the timeline(such as the compaction or clustering services) would + // mistakenly recognize the pending file as a pending operation, + // then all kinds of weird bugs occur. + HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); + if (!pendingInstants.isEmpty()) { + context.foreach( + pendingInstants, + instant -> activeTimeline.deleteInstantFileIfExists(instant), + Math.min(pendingInstants.size(), config.getArchiveDeleteParallelism()) + ); + } + if (!completedInstants.isEmpty()) { + // Due to the concurrency between deleting completed instants and reading data, + // there may be hole in the timeline, which can lead to errors when reading data. + // Therefore, the concurrency of deleting completed instants is temporarily disabled, + // and instants are deleted in ascending order to prevent the occurrence of such holes. + // See HUDI-7207 and #10325. + completedInstants.stream() + .forEach(instant -> activeTimeline.deleteInstantFileIfExists(instant)); + } + + return true; + } + + private void deleteAnyLeftOverMarkers(HoodieEngineContext context, ActiveAction activeAction) { + WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, activeAction.getInstantTime()); + if (writeMarkers.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) { + LOG.info("Cleaned up left over marker directory for instant: {}", activeAction); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java index 2e22c59238ef..9580ef262356 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/ConcurrentOperation.java @@ -63,13 +63,13 @@ public class ConcurrentOperation { public ConcurrentOperation(HoodieInstant instant, HoodieTableMetaClient metaClient) throws IOException { // Replace compaction.inflight to compaction.request since inflight does not contain compaction plan. if (instant.getAction().equals(COMPACTION_ACTION) && instant.getState().equals(HoodieInstant.State.INFLIGHT)) { - instant = new HoodieInstant(HoodieInstant.State.REQUESTED, COMPACTION_ACTION, instant.getTimestamp()); + instant = metaClient.createNewInstant(HoodieInstant.State.REQUESTED, COMPACTION_ACTION, instant.requestedTime()); } this.metadataWrapper = new HoodieMetadataWrapper(MetadataConversionUtils.createMetaWrapper(instant, metaClient)); this.commitMetadataOption = Option.empty(); this.actionState = instant.getState().name(); this.actionType = instant.getAction(); - this.instantTime = instant.getTimestamp(); + this.instantTime = instant.requestedTime(); init(instant); } @@ -78,7 +78,7 @@ public ConcurrentOperation(HoodieInstant instant, HoodieCommitMetadata commitMet this.metadataWrapper = new HoodieMetadataWrapper(commitMetadata); this.actionState = instant.getState().name(); this.actionType = instant.getAction(); - this.instantTime = instant.getTimestamp(); + this.instantTime = instant.requestedTime(); init(instant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java index 00c27127a50f..62d29fbf5802 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/DirectMarkerTransactionManager.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieNotSupportedException; @@ -45,21 +46,21 @@ public DirectMarkerTransactionManager(HoodieWriteConfig config, HoodieStorage st this.filePath = partitionPath + "/" + fileId; } - public void beginTransaction(String newTxnOwnerInstantTime) { + public void beginTransaction(String newTxnOwnerInstantTime, InstantGenerator instantGenerator) { if (isLockRequired) { LOG.info("Transaction starting for " + newTxnOwnerInstantTime + " and " + filePath); lockManager.lock(); - reset(currentTxnOwnerInstant, Option.of(getInstant(newTxnOwnerInstantTime)), Option.empty()); + reset(currentTxnOwnerInstant, Option.of(getInstant(newTxnOwnerInstantTime, instantGenerator)), Option.empty()); LOG.info("Transaction started for " + newTxnOwnerInstantTime + " and " + filePath); } } - public void endTransaction(String currentTxnOwnerInstantTime) { + public void endTransaction(String currentTxnOwnerInstantTime, InstantGenerator instantGenerator) { if (isLockRequired) { LOG.info("Transaction ending with transaction owner " + currentTxnOwnerInstantTime + " for " + filePath); - if (reset(Option.of(getInstant(currentTxnOwnerInstantTime)), Option.empty(), Option.empty())) { + if (reset(Option.of(getInstant(currentTxnOwnerInstantTime, instantGenerator)), Option.empty(), Option.empty())) { lockManager.unlock(); LOG.info("Transaction ended with transaction owner " + currentTxnOwnerInstantTime + " for " + filePath); @@ -85,7 +86,7 @@ private static TypedProperties createUpdatedLockProps( return props; } - private HoodieInstant getInstant(String instantTime) { - return new HoodieInstant(HoodieInstant.State.INFLIGHT, EMPTY_STRING, instantTime); + private HoodieInstant getInstant(String instantTime, InstantGenerator instantGenerator) { + return instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, EMPTY_STRING, instantTime); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/PreferWriterConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/PreferWriterConflictResolutionStrategy.java index 3a013b6e78d7..b25f43fe715b 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/PreferWriterConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/transaction/PreferWriterConflictResolutionStrategy.java @@ -54,7 +54,7 @@ public class PreferWriterConflictResolutionStrategy public Stream getCandidateInstants(HoodieTableMetaClient metaClient, HoodieInstant currentInstant, Option lastSuccessfulInstant) { HoodieActiveTimeline activeTimeline = metaClient.reloadActiveTimeline(); - if (ClusteringUtils.isClusteringInstant(activeTimeline, currentInstant) + if (ClusteringUtils.isClusteringInstant(activeTimeline, currentInstant, metaClient.getInstantGenerator()) || COMPACTION_ACTION.equals(currentInstant.getAction())) { return getCandidateInstantsForTableServicesCommits(activeTimeline, currentInstant); } else { @@ -72,7 +72,7 @@ private Stream getCandidateInstantsForNonTableServicesCommits(Hoo List completedCommitsInstants = activeTimeline .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) .filterCompletedInstants() - .findInstantsModifiedAfterByCompletionTime(currentInstant.getTimestamp()) + .findInstantsModifiedAfterByCompletionTime(currentInstant.requestedTime()) .getInstantsOrderedByCompletionTime() .collect(Collectors.toList()); LOG.info(String.format("Instants that may have conflict with %s are %s", currentInstant, completedCommitsInstants)); @@ -91,7 +91,7 @@ private Stream getCandidateInstantsForTableServicesCommits(Hoodie activeTimeline .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, COMPACTION_ACTION, DELTA_COMMIT_ACTION)) .filterCompletedInstants() - .findInstantsModifiedAfterByCompletionTime(currentInstant.getTimestamp()) + .findInstantsModifiedAfterByCompletionTime(currentInstant.requestedTime()) .getInstantsAsStream(); // Fetch list of ingestion inflight commits. 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 3c2c0d21e3ed..a4e05d2d6f7b 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 @@ -38,6 +38,9 @@ import java.util.Set; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * This class is a basic implementation of a conflict resolution strategy for concurrent writes {@link ConflictResolutionStrategy}. */ @@ -58,14 +61,14 @@ public Stream getCandidateInstants(HoodieTableMetaClient metaClie Stream completedCommitsInstantStream = activeTimeline .getCommitsTimeline() .filterCompletedInstants() - .findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().getTimestamp() : HoodieTimeline.INIT_INSTANT_TS) + .findInstantsAfter(lastSuccessfulInstant.isPresent() ? lastSuccessfulInstant.get().requestedTime() : HoodieTimeline.INIT_INSTANT_TS) .getInstantsAsStream(); Stream compactionAndClusteringPendingTimeline = activeTimeline .filterPendingReplaceClusteringAndCompactionTimeline() - .filter(instant -> ClusteringUtils.isClusteringInstant(activeTimeline, instant) + .filter(instant -> ClusteringUtils.isClusteringInstant(activeTimeline, instant, metaClient.getInstantGenerator()) || HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction())) - .findInstantsAfter(currentInstant.getTimestamp()) + .findInstantsAfter(currentInstant.requestedTime()) .getInstantsAsStream(); return Stream.concat(completedCommitsInstantStream, compactionAndClusteringPendingTimeline); } @@ -96,7 +99,7 @@ public Option resolveConflict(HoodieTable table, // supported for CLUSTER (https://issues.apache.org/jira/browse/HUDI-1042), // add that to the below check so that concurrent updates do not conflict. if (otherOperation.getOperationType() == WriteOperationType.COMPACT) { - if (HoodieTimeline.compareTimestamps(otherOperation.getInstantTimestamp(), HoodieTimeline.LESSER_THAN, thisOperation.getInstantTimestamp())) { + if (compareTimestamps(otherOperation.getInstantTimestamp(), LESSER_THAN, thisOperation.getInstantTimestamp())) { return thisOperation.getCommitMetadataOption(); } } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(thisOperation.getInstantActionType())) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java index 3a6d2509ad9b..a0de92755859 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java @@ -36,8 +36,8 @@ import java.text.ParseException; import java.time.Instant; -import static org.apache.hudi.common.table.timeline.HoodieActiveTimeline.NOT_PARSABLE_TIMESTAMPS; -import static org.apache.hudi.common.table.timeline.HoodieActiveTimeline.parseDateFromInstantTime; +import static org.apache.hudi.common.table.timeline.TimelineUtils.NOT_PARSABLE_TIMESTAMPS; +import static org.apache.hudi.common.table.timeline.TimelineUtils.parseDateFromInstantTime; import static org.apache.hudi.config.HoodieArchivalConfig.MAX_COMMITS_TO_KEEP; import static org.apache.hudi.config.HoodieArchivalConfig.MIN_COMMITS_TO_KEEP; import static org.apache.hudi.config.HoodieCleanConfig.CLEANER_COMMITS_RETAINED; @@ -73,7 +73,7 @@ public static Pair getMinAndMaxInstantsToKeep(HoodieTable getEarliestCommitToRetain( cleanerPolicy, cleanerCommitsRetained, latestCommit.isPresent() - ? parseDateFromInstantTime(latestCommit.get().getTimestamp()).toInstant() + ? parseDateFromInstantTime(latestCommit.get().requestedTime()).toInstant() : Instant.now(), cleanerHoursRetained, metaClient.getTableConfig().getTimelineTimezone()); } catch (ParseException e) { - if (NOT_PARSABLE_TIMESTAMPS.stream().noneMatch(ts -> latestCommit.get().getTimestamp().startsWith(ts))) { - LOG.warn("Error parsing instant time: " + latestCommit.get().getTimestamp()); + if (NOT_PARSABLE_TIMESTAMPS.stream().noneMatch(ts -> latestCommit.get().requestedTime().startsWith(ts))) { + LOG.warn("Error parsing instant time: " + latestCommit.get().requestedTime()); } } return earliestCommitToRetain; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java index 73576efda268..78e07fea16c3 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/DeletePartitionUtils.java @@ -64,7 +64,7 @@ public static void checkForPendingTableServiceActions(HoodieTable table, List partitionsToDrop.contains(fgIdInstantPair.getLeft().getPartitionPath())) - .forEach(x -> instantsOfOffendingPendingTableServiceAction.add(x.getRight().getTimestamp())); + .forEach(x -> instantsOfOffendingPendingTableServiceAction.add(x.getRight().requestedTime())); if (instantsOfOffendingPendingTableServiceAction.size() > 0) { throw new HoodieDeletePartitionPendingTableServiceException("Failed to drop partitions. " diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java index 9c092e2feccf..f8fa0d32a505 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.client.timeline.ActiveActionWithDetails; import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -32,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.Pair; @@ -86,7 +86,7 @@ public ClosableIterator getActiveActionsIterator() { * Reads the avro record for instant and details. */ private Pair> readInstant(GenericRecord record) { - final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString(); + final String instantTime = record.get(HoodieTableMetaClient.COMMIT_TIME_KEY).toString(); final String action = record.get(ACTION_TYPE_KEY).toString(); final String stateTransitionTime = (String) record.get(STATE_TRANSITION_TIME); final Option details = getMetadataKey(action).map(key -> { @@ -100,7 +100,8 @@ private Pair> readInstant(GenericRecord record) { } return null; }); - HoodieInstant instant = new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, + InstantGenerator instantGenerator = metaClient.getInstantGenerator(); + HoodieInstant instant = instantGenerator.createNewInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, instantTime, stateTransitionTime); return Pair.of(instant,details); } @@ -165,13 +166,13 @@ public boolean hasNext() { String lastInstantTime = null; if (nextInstantAndDetail != null) { instantAndDetails.add(nextInstantAndDetail); - lastInstantTime = nextInstantAndDetail.getKey().getTimestamp(); + lastInstantTime = nextInstantAndDetail.getKey().requestedTime(); nextInstantAndDetail = null; } while (itr.hasNext()) { HoodieRecord record = itr.next(); Pair> instantAndDetail = readInstant((GenericRecord) record.getData()); - String instantTime = instantAndDetail.getKey().getTimestamp(); + String instantTime = instantAndDetail.getKey().requestedTime(); if (filter == null || filter.isInRange(instantTime)) { if (lastInstantTime == null) { instantAndDetails.add(instantAndDetail); 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 7c82682f8504..6e7bf12867d1 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 @@ -142,7 +142,7 @@ public static Set getInflightAndRequestedInstants(HoodieTableMetaClient .getTimelineOfActions(timelineActions) .filterInflightsAndRequested() .getInstantsAsStream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .collect(Collectors.toSet()); } @@ -155,6 +155,6 @@ public static Stream getCompletedInstantsDuringCurrentWriteOperat .getCommitsTimeline() .filterCompletedInstants() .getInstantsAsStream() - .filter(f -> pendingInstants.contains(f.getTimestamp())); + .filter(f -> pendingInstants.contains(f.requestedTime())); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 3aeaef5973c8..0dfcc66833bf 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -234,7 +234,9 @@ public class HoodieWriteConfig extends HoodieConfig { public static final ConfigProperty TIMELINE_LAYOUT_VERSION_NUM = ConfigProperty .key("hoodie.timeline.layout.version") .defaultValue(Integer.toString(TimelineLayoutVersion.CURR_VERSION)) - .withValidValues(Integer.toString(TimelineLayoutVersion.VERSION_0), Integer.toString(TimelineLayoutVersion.VERSION_1)) + .withValidValues(Integer.toString(TimelineLayoutVersion.VERSION_0), + Integer.toString(TimelineLayoutVersion.VERSION_1), + Integer.toString(TimelineLayoutVersion.VERSION_2)) .markAdvanced() .sinceVersion("0.5.1") .withDocumentation("Controls the layout of the timeline. Version 0 relied on renames, Version 1 (default) models " @@ -2872,8 +2874,9 @@ public Builder withSchemaEvolutionEnable(boolean enable) { } public Builder withWriteTableVersion(int writeVersion) { - writeConfig.setValue(WRITE_TABLE_VERSION, String.valueOf(HoodieTableVersion.fromVersionCode(writeVersion).versionCode())); - return this; + HoodieTableVersion tableVersion = HoodieTableVersion.fromVersionCode(writeVersion); + writeConfig.setValue(WRITE_TABLE_VERSION, String.valueOf(tableVersion.versionCode())); + return withTimelineLayoutVersion(tableVersion.getTimelineLayoutVersion().getVersion()); } public Builder withAutoUpgradeVersion(boolean enable) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java index 918d4be7fcd3..8b491e168ebe 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/HoodieIndexUtils.java @@ -92,7 +92,7 @@ public static List getLatestBaseFilesForPartition(String partiti .filterCompletedInstants().lastInstant(); if (latestCommitTime.isPresent()) { return hoodieTable.getBaseFileOnlyView() - .getLatestBaseFilesBeforeOrOn(partition, latestCommitTime.get().getTimestamp()) + .getLatestBaseFilesBeforeOrOn(partition, latestCommitTime.get().requestedTime()) .collect(toList()); } return Collections.emptyList(); @@ -112,7 +112,7 @@ public static List getLatestFileSlicesForPartition( .filterCompletedInstants().lastInstant(); if (latestCommitTime.isPresent()) { return hoodieTable.getHoodieView() - .getLatestFileSlicesBeforeOrOn(partition, latestCommitTime.get().getTimestamp(), true) + .getLatestFileSlicesBeforeOrOn(partition, latestCommitTime.get().requestedTime(), true) .collect(toList()); } return Collections.emptyList(); @@ -244,7 +244,7 @@ private static HoodieData> getExistingRecords( .getActiveTimeline() // we need to include all actions and completed .filterCompletedInstants() .lastInstant() - .map(HoodieInstant::getTimestamp); + .map(HoodieInstant::requestedTime); return partitionLocations.flatMap(p -> new HoodieMergedReadHandle(config, instantTime, hoodieTable, Pair.of(p.getKey(), p.getValue())) .getMergedRecords().iterator()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java index 298403250a23..d72eca3c4096 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/index/bucket/HoodieSimpleBucketIndex.java @@ -69,7 +69,7 @@ public Map loadBucketIdToFileIdMappingForPartitio // bucketId -> fileIds Map bucketIdToFileIdMapping = new HashMap<>(); HoodieActiveTimeline hoodieActiveTimeline = hoodieTable.getMetaClient().reloadActiveTimeline(); - Set pendingInstants = hoodieActiveTimeline.filterInflights().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + Set pendingInstants = hoodieActiveTimeline.filterInflights().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); HoodieIndexUtils .getLatestFileSlicesForPartition(partition, hoodieTable) 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 bf930ac9fce7..6d766403e5cc 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 @@ -54,6 +54,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CompactionUtils; @@ -101,8 +102,8 @@ import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.getIndexInflightInstant; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeIndexPlan; import static org.apache.hudi.metadata.HoodieMetadataWriteUtils.createMetadataWriteConfig; import static org.apache.hudi.metadata.HoodieTableMetadata.METADATA_TABLE_NAME_SUFFIX; @@ -267,7 +268,7 @@ protected boolean initializeIfNeeded(HoodieTableMetaClient dataMetaClient, // If there is no commit on the dataset yet, use the SOLO_COMMIT_TIMESTAMP as the instant time for initial commit // Otherwise, we use the timestamp of the latest completed action. - String initializationTime = dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + String initializationTime = dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); initializeFromFilesystem(initializationTime, metadataPartitionsToInit, inflightInstantTimestamp); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); return true; @@ -615,7 +616,7 @@ private Pair> initializeSecondaryIndexPartitio private List> getPartitionFileSlicePairs() throws IOException { String latestInstant = dataMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant() - .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + .map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); try (HoodieMetadataFileSystemView fsView = getMetadataView()) { // Collect the list of latest file slices present in each partition List partitions = metadata.getAllPartitionPaths(); @@ -658,7 +659,7 @@ private Pair> initializeRecordIndexPartition() this.getClass().getSimpleName()); } else { final List> partitionFileSlicePairs = new ArrayList<>(); - String latestCommit = dataMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant().map(instant -> instant.getTimestamp()).orElse(SOLO_COMMIT_TIMESTAMP); + String latestCommit = dataMetaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant().map(instant -> instant.requestedTime()).orElse(SOLO_COMMIT_TIMESTAMP); for (String partition : partitions) { fsView.getLatestMergedFileSlicesBeforeOrOn(partition, latestCommit).forEach(fs -> partitionFileSlicePairs.add(Pair.of(partition, fs))); } @@ -712,7 +713,7 @@ private static HoodieData readRecordKeysFromFileSliceSnapshot(Hood Option instantTime = metaClient.getActiveTimeline().getCommitsTimeline() .filterCompletedInstants() .lastInstant() - .map(HoodieInstant::getTimestamp); + .map(HoodieInstant::requestedTime); engineContext.setJobStatus(activeModule, "Record Index: reading record keys from " + partitionFileSlicePairs.size() + " file slices"); final int parallelism = Math.min(partitionFileSlicePairs.size(), recordIndexMaxParallelism); @@ -764,7 +765,7 @@ private Set getPendingDataInstants(HoodieTableMetaClient dataMetaClient) .getInstantsAsStream().filter(i -> !i.isCompleted()) // regular writers should not be blocked due to pending indexing action .filter(i -> !HoodieTimeline.INDEXING_ACTION.equals(i.getAction())) - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .collect(Collectors.toSet()); } @@ -956,6 +957,7 @@ public void dropMetadataPartitions(List metadataPartitions) throws IOExc * if the partition path in the plan matches with the given partition path. */ private static void deletePendingIndexingInstant(HoodieTableMetaClient metaClient, String partitionPath) { + InstantGenerator instantGenerator = metaClient.getInstantGenerator(); metaClient.reloadActiveTimeline().filterPendingIndexTimeline().getInstantsAsStream().filter(instant -> REQUESTED.equals(instant.getState())) .forEach(instant -> { try { @@ -963,7 +965,7 @@ private static void deletePendingIndexingInstant(HoodieTableMetaClient metaClien if (indexPlan.getIndexPartitionInfos().stream() .anyMatch(indexPartitionInfo -> indexPartitionInfo.getMetadataPartitionPath().equals(partitionPath))) { metaClient.getActiveTimeline().deleteInstantFileIfExists(instant); - metaClient.getActiveTimeline().deleteInstantFileIfExists(getIndexInflightInstant(instant.getTimestamp())); + metaClient.getActiveTimeline().deleteInstantFileIfExists(instantGenerator.getIndexInflightInstant(instant.requestedTime())); } } catch (IOException e) { LOG.error("Failed to delete the instant file corresponding to {}", instant); @@ -1230,8 +1232,9 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { dataMetaClient.reloadActiveTimeline(); // Fetch the commit to restore to (savepointed commit time) - HoodieInstant restoreInstant = new HoodieInstant(REQUESTED, HoodieTimeline.RESTORE_ACTION, instantTime); - HoodieInstant requested = HoodieTimeline.getRestoreRequestedInstant(restoreInstant); + InstantGenerator datainstantGenerator = dataMetaClient.getInstantGenerator(); + HoodieInstant restoreInstant = datainstantGenerator.createNewInstant(REQUESTED, HoodieTimeline.RESTORE_ACTION, instantTime); + HoodieInstant requested = datainstantGenerator.getRestoreRequestedInstant(restoreInstant); HoodieRestorePlan restorePlan = null; try { restorePlan = TimelineMetadataUtils.deserializeAvroMetadata( @@ -1246,7 +1249,7 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { List filesGroups = metadata.getMetadataFileSystemView().getAllFileGroups(FILES.getPartitionPath()).collect(Collectors.toList()); boolean cannotRestore = filesGroups.stream().map(fileGroup -> fileGroup.getAllFileSlices().map(FileSlice::getBaseInstantTime).anyMatch( - instantTime1 -> HoodieTimeline.compareTimestamps(instantTime1, LESSER_THAN_OR_EQUALS, restoreToInstantTime))).anyMatch(canRestore -> !canRestore); + instantTime1 -> compareTimestamps(instantTime1, LESSER_THAN_OR_EQUALS, restoreToInstantTime))).anyMatch(canRestore -> !canRestore); if (cannotRestore) { throw new HoodieMetadataException(String.format("Can't restore to %s since there is no base file in MDT lesser than the commit to restore to. " + "Please delete metadata table and retry", restoreToInstantTime)); @@ -1295,7 +1298,8 @@ public void update(HoodieRollbackMetadata rollbackMetadata, String instantTime) // The commit which is being rolled back on the dataset final String commitToRollbackInstantTime = rollbackMetadata.getCommitsRollback().get(0); // The deltacommit that will be rolled back - HoodieInstant deltaCommitInstant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, commitToRollbackInstantTime); + HoodieInstant deltaCommitInstant = metadataMetaClient.createNewInstant(HoodieInstant.State.COMPLETED, + HoodieTimeline.DELTA_COMMIT_ACTION, commitToRollbackInstantTime); if (metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().containsInstant(deltaCommitInstant)) { validateRollback(commitToRollbackInstantTime); LOG.info("Rolling back MDT deltacommit {}", commitToRollbackInstantTime); @@ -1324,7 +1328,7 @@ private void validateRollback(String commitToRollbackInstantTime) { // Compaction on MDT only occurs when all actions are completed on the dataset. // Hence, this case implies a rollback of completed commit which should actually be handled using restore. if (compactionInstant.getAction().equals(COMMIT_ACTION)) { - final String compactionInstantTime = compactionInstant.getTimestamp(); + final String compactionInstantTime = compactionInstant.requestedTime(); if (commitToRollbackInstantTime.length() == compactionInstantTime.length() && LESSER_THAN_OR_EQUALS.test(commitToRollbackInstantTime, compactionInstantTime)) { throw new HoodieMetadataException( String.format("Commit being rolled back %s is earlier than the latest compaction %s. There are %d deltacommits after this compaction: %s", @@ -1388,7 +1392,7 @@ protected void commitInternal(String instantTime, Map alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)) + Option alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.requestedTime().equals(instantTime)) .lastInstant(); LOG.info("{} completed commit at {} being applied to MDT.", alreadyCompletedInstant.isPresent() ? "Already" : "Partially", instantTime); @@ -1508,7 +1512,7 @@ public void performTableServices(Option inFlightInstantTimestamp) { cleanIfNecessary(writeClient); // Do timeline validation before scheduling compaction/logCompaction operations. if (validateCompactionScheduling()) { - String latestDeltacommitTime = lastInstant.get().getTimestamp(); + String latestDeltacommitTime = lastInstant.get().requestedTime(); LOG.info("Latest deltacommit time found is {}, running compaction operations.", latestDeltacommitTime); compactIfNecessary(writeClient); } @@ -1553,9 +1557,9 @@ protected void compactIfNecessary(BaseHoodieWriteClient writeClient) { final String compactionInstantTime = dataMetaClient.reloadActiveTimeline() // The filtering strategy is kept in line with the rollback premise, if an instant is pending on DT but completed on MDT, // generates a compaction time smaller than it so that the instant could then been rolled back. - .filterInflightsAndRequested().filter(instant -> metadataCompletedTimeline.containsInstant(instant.getTimestamp())).firstInstant() + .filterInflightsAndRequested().filter(instant -> metadataCompletedTimeline.containsInstant(instant.requestedTime())).firstInstant() // minus the pending instant time by 1 millisecond to avoid conflicts on the MDT. - .map(instant -> HoodieInstantTimeGenerator.instantTimeMinusMillis(instant.getTimestamp(), 1L)) + .map(instant -> HoodieInstantTimeGenerator.instantTimeMinusMillis(instant.requestedTime(), 1L)) .orElse(writeClient.createNewInstantTime(false)); // we need to avoid checking compaction w/ same instant again. @@ -1584,7 +1588,7 @@ protected void cleanIfNecessary(BaseHoodieWriteClient writeClient) { .getCommitAndReplaceTimeline().filterCompletedInstants().lastInstant(); if (lastCompletedCompactionInstant.isPresent() && metadataMetaClient.getActiveTimeline().filterCompletedInstants() - .findInstantsAfter(lastCompletedCompactionInstant.get().getTimestamp()).countInstants() < 3) { + .findInstantsAfter(lastCompletedCompactionInstant.get().requestedTime()).countInstants() < 3) { // do not clean the log files immediately after compaction to give some buffer time for metadata table reader, // because there is case that the reader has prepared for the log file readers already before the compaction completes // while before/during the reading of the log files, the cleaning triggers and delete the reading files, diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java index 1c9cdda2ba28..eeade79aefcc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataWriteUtils.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.marker.MarkerType; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCleanConfig; @@ -109,7 +108,6 @@ public static HoodieWriteConfig createMetadataWriteConfig( // Create the write config for the metadata table by borrowing options from the main write config. HoodieWriteConfig.Builder builder = HoodieWriteConfig.newBuilder() .withEngineType(writeConfig.getEngineType()) - .withTimelineLayoutVersion(TimelineLayoutVersion.CURR_VERSION) .withWriteTableVersion(writeConfig.getWriteVersion().versionCode()) .withMergeAllowDuplicateOnInserts(false) .withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder() 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 cb5f04cca026..a2c56e522597 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 @@ -51,6 +51,9 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.SyncableFileSystemView; @@ -131,6 +134,9 @@ public abstract class HoodieTable implements Serializable { protected final TaskContextSupplier taskContextSupplier; private transient HoodieTableMetadata metadata; private transient HoodieStorageLayout storageLayout; + private final InstantGenerator instantGenerator; + private final InstantFileNameGenerator instantFileNameGenerator; + private final InstantFileNameParser instantFileNameParser; private final boolean isMetadataTable; private transient FileSystemViewManager viewManager; @@ -140,6 +146,10 @@ protected HoodieTable(HoodieWriteConfig config, HoodieEngineContext context, Hoo this.config = config; this.context = context; this.isMetadataTable = HoodieTableMetadata.isMetadataTable(config.getBasePath()); + this.instantGenerator = metaClient.getInstantGenerator(); + this.instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); + this.instantFileNameParser = metaClient.getInstantFileNameParser(); + this.viewManager = getViewManager(); this.metaClient = metaClient; this.taskContextSupplier = context.getTaskContextSupplier(); } @@ -292,6 +302,18 @@ public HoodieTableMetaClient getMetaClient() { return metaClient; } + public InstantGenerator getInstantGenerator() { + return instantGenerator; + } + + public InstantFileNameGenerator getInstantFileNameGenerator() { + return instantFileNameGenerator; + } + + public InstantFileNameParser getInstantFileNameParser() { + return instantFileNameParser; + } + /** * @return if the table is physically partitioned, based on the partition fields stored in the table config. */ @@ -395,7 +417,7 @@ public HoodieTimeline getCompletedSavepointTimeline() { * Get the list of savepoint timestamps in this table. */ public Set getSavepointTimestamps() { - return getCompletedSavepointTimeline().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + return getCompletedSavepointTimeline().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); } public HoodieActiveTimeline getActiveTimeline() { @@ -649,7 +671,7 @@ public void rollbackInflightClustering(HoodieInstant inflightInstant, rollbackInflightInstant(inflightInstant, getPendingRollbackInstantFunc); if (deleteInstants) { // above rollback would still keep requested in the timeline. so, lets delete it if if are looking to purge the pending clustering fully. - getActiveTimeline().deletePending(new HoodieInstant(HoodieInstant.State.REQUESTED, inflightInstant.getAction(), inflightInstant.getTimestamp())); + getActiveTimeline().deletePending(instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, inflightInstant.getAction(), inflightInstant.requestedTime())); } } @@ -661,8 +683,8 @@ public void rollbackInflightClustering(HoodieInstant inflightInstant, */ private void rollbackInflightInstant(HoodieInstant inflightInstant, Function> getPendingRollbackInstantFunc) { - final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry - -> entry.getRollbackInstant().getTimestamp()) + final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.requestedTime()).map(entry + -> entry.getRollbackInstant().requestedTime()) .orElseGet(() -> getMetaClient().createNewInstantTime()); scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers(), false); @@ -677,8 +699,8 @@ private void rollbackInflightInstant(HoodieInstant inflightInstant, * @param inflightInstant Inflight Compaction Instant */ public void rollbackInflightLogCompaction(HoodieInstant inflightInstant, Function> getPendingRollbackInstantFunc) { - final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.getTimestamp()).map(entry - -> entry.getRollbackInstant().getTimestamp()) + final String commitTime = getPendingRollbackInstantFunc.apply(inflightInstant.requestedTime()).map(entry + -> entry.getRollbackInstant().requestedTime()) .orElseGet(() -> getMetaClient().createNewInstantTime()); scheduleRollback(context, commitTime, inflightInstant, false, config.shouldRollbackUsingMarkers(), false); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java index d0a7acd676c0..c4ca56778328 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/BaseActionExecutor.java @@ -26,6 +26,9 @@ import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -46,7 +49,9 @@ public abstract class BaseActionExecutor implements Serializable protected final HoodieWriteConfig config; protected final HoodieTable table; - + protected final InstantGenerator instantGenerator; + protected final InstantFileNameParser instantFileNameParser; + protected final InstantFileNameGenerator instantFileNameGenerator; protected final String instantTime; public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, HoodieTable table, String instantTime) { @@ -54,6 +59,9 @@ public BaseActionExecutor(HoodieEngineContext context, HoodieWriteConfig config, this.storageConf = context.getStorageConf(); this.config = config; this.table = table; + this.instantGenerator = table.getInstantGenerator(); + this.instantFileNameGenerator = table.getInstantFileNameGenerator(); + this.instantFileNameParser = table.getInstantFileNameParser(); this.instantTime = instantTime; } 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 c44421e9ce21..9b9e93cf3c6e 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 @@ -174,7 +174,7 @@ List clean(HoodieEngineContext context, HoodieCleanerPlan clean return HoodieCleanStat.newBuilder().withPolicy(config.getCleanerPolicy()).withPartitionPath(partitionPath) .withEarliestCommitRetained(Option.ofNullable( actionInstant != null - ? new HoodieInstant(HoodieInstant.State.valueOf(actionInstant.getState()), + ? instantGenerator.createNewInstant(HoodieInstant.State.valueOf(actionInstant.getState()), actionInstant.getAction(), actionInstant.getTimestamp()) : null)) .withLastCompletedCommitTimestamp(cleanerPlan.getLastCompletedCommitTimestamp()) @@ -223,7 +223,7 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan table.getMetaClient().reloadActiveTimeline(); HoodieCleanMetadata metadata = CleanerUtils.convertCleanMetadata( - inflightInstant.getTimestamp(), + inflightInstant.requestedTime(), Option.of(timer.endTimer()), cleanStats, cleanerPlan.getExtraMetadata() @@ -231,10 +231,10 @@ private HoodieCleanMetadata runClean(HoodieTable table, HoodieInstan if (!skipLocking) { this.txnManager.beginTransaction(Option.of(inflightInstant), Option.empty()); } - writeTableMetadata(metadata, inflightInstant.getTimestamp()); + writeTableMetadata(metadata, inflightInstant.requestedTime()); table.getActiveTimeline().transitionCleanInflightToComplete(false, inflightInstant, TimelineMetadataUtils.serializeCleanMetadata(metadata)); - LOG.info("Marked clean started on " + inflightInstant.getTimestamp() + " as complete"); + LOG.info("Marked clean started on " + inflightInstant.requestedTime() + " as complete"); return metadata; } catch (IOException e) { throw new HoodieIOException("Failed to clean up after commit", e); @@ -255,7 +255,7 @@ public HoodieCleanMetadata execute() { // try to clean old history schema. try { FileBasedInternalSchemaStorageManager fss = new FileBasedInternalSchemaStorageManager(table.getMetaClient()); - fss.cleanOldFiles(pendingCleanInstants.stream().map(is -> is.getTimestamp()).collect(Collectors.toList())); + fss.cleanOldFiles(pendingCleanInstants.stream().map(is -> is.requestedTime()).collect(Collectors.toList())); } catch (Exception e) { // we should not affect original clean logic. Swallow exception and log warn. LOG.warn("failed to clean old history schema"); @@ -290,7 +290,7 @@ public HoodieCleanMetadata execute() { private void checkIfOtherWriterCommitted(HoodieInstant hoodieInstant, HoodieIOException e) { table.getMetaClient().reloadActiveTimeline(); - if (table.getCleanTimeline().filterCompletedInstants().containsInstant(hoodieInstant.getTimestamp())) { + if (table.getCleanTimeline().filterCompletedInstants().containsInstant(hoodieInstant.requestedTime())) { LOG.warn("Clean operation was completed by another writer for instant: " + hoodieInstant); } else { LOG.error("Failed to perform previous clean operation, instant: " + hoodieInstant, e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java index c2baccf9e6ea..074a87cc6351 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/clean/CleanPlanActionExecutor.java @@ -116,7 +116,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { } LOG.info( "Earliest commit to retain for clean : {}", - earliestInstant.isPresent() ? earliestInstant.get().getTimestamp() : "null"); + earliestInstant.isPresent() ? earliestInstant.get().requestedTime() : "null"); LOG.info( "Total partitions to clean : {}, with policy {}", partitionsToClean.size(), @@ -152,7 +152,7 @@ HoodieCleanerPlan requestClean(HoodieEngineContext context) { } return new HoodieCleanerPlan( - earliestInstant.map(x -> new HoodieActionInstant(x.getTimestamp(), x.getAction(), x.getState().name())).orElse(null), + earliestInstant.map(x -> new HoodieActionInstant(x.requestedTime(), x.getAction(), x.getState().name())).orElse(null), planner.getLastCompletedCommitTimestamp(), // Note: This is the start time of the last completed ingestion before this clean. config.getCleanerPolicy().name(), Collections.emptyMap(), @@ -188,7 +188,7 @@ protected Option requestClean(String startCleanTime) { if (nonEmpty(cleanerPlan.getFilePathsToBeDeletedPerPartition()) && cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().mapToInt(List::size).sum() > 0) { // Only create cleaner plan which does some work - final HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime); + final HoodieInstant cleanInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startCleanTime); // Save to both aux and timeline folder try { table.getActiveTimeline().saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); 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 c7c7bed20644..893dfe8548a4 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 @@ -64,6 +64,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * Cleaner is responsible for garbage collecting older files in a given partition path. Such that *

@@ -135,7 +140,7 @@ private HoodieSavepointMetadata getSavepointMetadata(String savepointTimestamp) throw new HoodieSavepointException( "Could not get data files for savepoint " + savepointTimestamp + ". No such savepoint."); } - HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTimestamp); + HoodieInstant instant = hoodieTable.getMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepointTimestamp); try { return TimelineMetadataUtils.deserializeHoodieSavepointMetadata( hoodieTable.getActiveTimeline().getInstantDetails(instant).get()); @@ -215,9 +220,9 @@ private List getPartitionPathsForIncrementalCleaning(HoodieCleanMetadata newInstantToRetain); 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())) + .filter(instant -> compareTimestamps(instant.requestedTime(), GREATER_THAN_OR_EQUALS, + cleanMetadata.getEarliestCommitToRetain()) && compareTimestamps(instant.requestedTime(), + LESSER_THAN, newInstantToRetain.get().requestedTime())) .flatMap(this::getPartitionsForInstants).distinct().collect(Collectors.toList()); } } @@ -250,8 +255,8 @@ private Stream getPartitionsForInstants(HoodieInstant instant) { hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); return Stream.concat(replaceCommitMetadata.getPartitionToReplaceFileIds().keySet().stream(), replaceCommitMetadata.getPartitionToWriteStats().keySet().stream()); } else { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), + HoodieCommitMetadata commitMetadata = hoodieTable.getMetaClient() + .getCommitMetadataSerDe().deserialize(instant, hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); return commitMetadata.getPartitionToWriteStats().keySet().stream(); } @@ -419,8 +424,8 @@ private Pair> getFilesToCleanKeepingLatestCommits(S } // Always keep the last commit - if (!isFileSliceNeededForPendingMajorOrMinorCompaction(aSlice) && HoodieTimeline - .compareTimestamps(earliestInstant.getTimestamp(), HoodieTimeline.GREATER_THAN, fileCommitTime)) { + if (!isFileSliceNeededForPendingMajorOrMinorCompaction(aSlice) + && compareTimestamps(earliestInstant.requestedTime(), GREATER_THAN, fileCommitTime)) { // this is a commit, that should be cleaned. aFile.ifPresent(hoodieDataFile -> { deletePaths.add(new CleanFileInfo(hoodieDataFile.getPath(), false)); @@ -441,7 +446,7 @@ private Pair> getFilesToCleanKeepingLatestCommits(S // mark it to be deleted if (fileGroups.isEmpty() && !hasPendingFiles(partitionPath) - && noSubsequentReplaceCommit(earliestInstant.getTimestamp(), partitionPath)) { + && noSubsequentReplaceCommit(earliestInstant.requestedTime(), partitionPath)) { toDeletePartition = true; } } @@ -487,7 +492,7 @@ private Pair> getFilesToCleanKeepingLatestHours(Str private List getReplacedFilesEligibleToClean(List savepointedFiles, String partitionPath, Option earliestCommitToRetain) { final Stream replacedGroups; if (earliestCommitToRetain.isPresent()) { - replacedGroups = hoodieTable.getHoodieView().getReplacedFileGroupsBefore(earliestCommitToRetain.get().getTimestamp(), partitionPath); + replacedGroups = hoodieTable.getHoodieView().getReplacedFileGroupsBefore(earliestCommitToRetain.get().requestedTime(), partitionPath); } else { replacedGroups = hoodieTable.getHoodieView().getAllReplacedFileGroups(partitionPath); } @@ -504,7 +509,7 @@ private List getReplacedFilesEligibleToClean(List savepoi private String getLatestVersionBeforeCommit(List fileSliceList, HoodieInstant instantTime) { for (FileSlice file : fileSliceList) { String fileCommitTime = file.getBaseInstantTime(); - if (HoodieTimeline.compareTimestamps(instantTime.getTimestamp(), HoodieTimeline.GREATER_THAN, fileCommitTime)) { + if (compareTimestamps(instantTime.requestedTime(), GREATER_THAN, fileCommitTime)) { // fileList is sorted on the reverse, so the first commit we find <= instantTime is the // one we want return fileCommitTime; @@ -574,7 +579,7 @@ public Option getEarliestCommitToRetain() { * Returns the last completed commit timestamp before clean. */ public String getLastCompletedCommitTimestamp() { - return getCommitTimeline().lastInstant().map(HoodieInstant::getTimestamp).orElse(""); + return getCommitTimeline().lastInstant().map(HoodieInstant::requestedTime).orElse(""); } /* @@ -596,7 +601,7 @@ private boolean isFileSliceNeededForPendingCompaction(FileSlice fileSlice) { CompactionOperation op = fgIdToPendingCompactionOperations.get(fileSlice.getFileGroupId()); if (null != op) { // If file slice's instant time is newer or same as that of operation, do not clean - return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), HoodieTimeline.GREATER_THAN_OR_EQUALS, op.getBaseInstantTime() + return compareTimestamps(fileSlice.getBaseInstantTime(), GREATER_THAN_OR_EQUALS, op.getBaseInstantTime() ); } return false; @@ -612,7 +617,7 @@ private boolean isFileSliceNeededForPendingLogCompaction(FileSlice fileSlice) { CompactionOperation op = fgIdToPendingLogCompactionOperations.get(fileSlice.getFileGroupId()); if (null != op) { // If file slice's instant time is newer or same as that of operation, do not clean - return HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), HoodieTimeline.GREATER_THAN_OR_EQUALS, op.getBaseInstantTime() + return compareTimestamps(fileSlice.getBaseInstantTime(), GREATER_THAN_OR_EQUALS, op.getBaseInstantTime() ); } return false; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java index e0f500467691..549c382d404f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/ClusteringPlanActionExecutor.java @@ -61,7 +61,7 @@ protected Option createClusteringPlan() { table.getActiveTimeline().getLastClusteringInstant(); int commitsSinceLastClustering = table.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::getTimestamp).orElse("0"), Integer.MAX_VALUE) + .findInstantsAfter(lastClusteringInstant.map(HoodieInstant::requestedTime).orElse("0"), Integer.MAX_VALUE) .countInstants(); if (config.inlineClusteringEnabled() && config.getInlineClusterMaxCommits() > commitsSinceLastClustering) { @@ -91,7 +91,7 @@ public Option execute() { Option planOption = createClusteringPlan(); if (planOption.isPresent()) { HoodieInstant clusteringInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, instantTime); + instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, instantTime); try { HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setOperationType(WriteOperationType.CLUSTER.name()) diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/util/ConsistentHashingUpdateStrategyUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/util/ConsistentHashingUpdateStrategyUtils.java index bb1b3124c951..6c3c2700f6c6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/util/ConsistentHashingUpdateStrategyUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/cluster/util/ConsistentHashingUpdateStrategyUtils.java @@ -67,7 +67,7 @@ public static Map> constructPar Map partitionToHashingMeta = new HashMap<>(); Map partitionToInstant = new HashMap<>(); for (Pair pair : instantPlanPairs) { - String instant = pair.getLeft().getTimestamp(); + String instant = pair.getLeft().requestedTime(); HoodieClusteringPlan plan = pair.getRight(); extractHashingMetadataFromClusteringPlan(instant, plan, table, partitions, partitionToHashingMeta, partitionToInstant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 91cd6f883b01..ea20983819f4 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -152,10 +153,10 @@ void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String insta HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); String commitActionType = getCommitActionType(); - HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); + HoodieInstant requested = table.getMetaClient().createNewInstant(State.REQUESTED, commitActionType, instantTime); activeTimeline.transitionRequestedToInflight( requested, - serializeCommitMetadata(metadata), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), metadata), config.shouldAllowMultiWriteOnSameInstant()); } catch (IOException io) { throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", io); @@ -188,7 +189,8 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { } protected void autoCommit(HoodieWriteMetadata result) { - final Option inflightInstant = Option.of(new HoodieInstant(State.INFLIGHT, + InstantGenerator factory = table.getMetaClient().getInstantGenerator(); + final Option inflightInstant = Option.of(factory.createNewInstant(State.INFLIGHT, getCommitActionType(), instantTime)); ValidationUtils.checkState(this.txnManagerOption.isPresent(), "The transaction manager has not been initialized"); TransactionManager txnManager = this.txnManagerOption.get(); @@ -224,8 +226,8 @@ protected void commit(HoodieData writeStatuses, HoodieWriteMetadata // cannot serialize maps with null values metadata.getExtraMetadata().entrySet().removeIf(entry -> entry.getValue() == null); activeTimeline.saveAsComplete(false, - new HoodieInstant(true, actionType, instantTime), - serializeCommitMetadata(metadata)); + table.getMetaClient().createNewInstant(State.INFLIGHT, actionType, instantTime), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), metadata)); LOG.info("Committed " + instantTime); result.setCommitMetadata(Option.of(metadata)); } catch (IOException e) { @@ -264,7 +266,8 @@ protected abstract Iterator> handleUpdate(String partitionPath protected HoodieWriteMetadata> executeClustering(HoodieClusteringPlan clusteringPlan) { context.setJobStatus(this.getClass().getSimpleName(), "Clustering records for " + config.getTableName()); - HoodieInstant instant = ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline()).get(); + HoodieInstant instant = ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline(), + table.getMetaClient().getInstantGenerator()).get(); // Mark instant as clustering inflight ClusteringUtils.transitionClusteringOrReplaceRequestedToInflight(instant, Option.empty(), table.getActiveTimeline()); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java index ab61ddbb314b..e81fd1cdfc26 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactHelpers.java @@ -27,7 +27,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieCompactionException; @@ -63,8 +63,9 @@ public static CompactHelpers getInstance() { public HoodieCommitMetadata createCompactionMetadata( HoodieTable table, String compactionInstantTime, HoodieData writeStatuses, String schema) throws IOException { + InstantGenerator instantGenerator = table.getInstantGenerator(); byte[] planBytes = table.getActiveTimeline().readCompactionPlanAsBytes( - HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get(); + instantGenerator.getCompactionRequestedInstant(compactionInstantTime)).get(); HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(planBytes); List updateStatusMap = writeStatuses.map(WriteStatus::getStat).collectAsList(); HoodieCommitMetadata metadata = new HoodieCommitMetadata(true); @@ -82,10 +83,11 @@ public HoodieCommitMetadata createCompactionMetadata( public void completeInflightCompaction(HoodieTable table, String compactionCommitTime, HoodieCommitMetadata commitMetadata) { HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); try { + InstantGenerator instantGenerator = table.getInstantGenerator(); // Callers should already guarantee the lock. activeTimeline.transitionCompactionInflightToComplete(false, - HoodieTimeline.getCompactionInflightInstant(compactionCommitTime), - serializeCommitMetadata(commitMetadata)); + instantGenerator.getCompactionInflightInstant(compactionCommitTime), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), commitMetadata)); } catch (IOException e) { throw new HoodieCompactionException( "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + compactionCommitTime, e); @@ -96,9 +98,10 @@ public void completeInflightLogCompaction(HoodieTable table, String logCompactio HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); try { // Callers should already guarantee the lock. + InstantGenerator instantGenerator = table.getInstantGenerator(); activeTimeline.transitionLogCompactionInflightToComplete(false, - HoodieTimeline.getLogCompactionInflightInstant(logCompactionCommitTime), - serializeCommitMetadata(commitMetadata)); + instantGenerator.getLogCompactionInflightInstant(logCompactionCommitTime), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), commitMetadata)); } catch (IOException e) { throw new HoodieCompactionException( "Failed to commit " + table.getMetaClient().getBasePath() + " at time " + logCompactionCommitTime, e); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java index bdb836161967..a64fb1d6dcf2 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/CompactionExecutionHelper.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieCompactionHandler; @@ -40,7 +39,7 @@ public class CompactionExecutionHelper i protected void transitionRequestedToInflight(HoodieTable table, String compactionInstantTime) { HoodieActiveTimeline timeline = table.getActiveTimeline(); - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant instant = table.getMetaClient().getInstantGenerator().getCompactionRequestedInstant(compactionInstantTime); // Mark instant as compaction inflight timeline.transitionCompactionRequestedToInflight(instant); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java index 9defec99c38e..e92c2a5d7c2d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/HoodieCompactor.java @@ -265,7 +265,7 @@ public String getMaxInstantTime(HoodieTableMetaClient metaClient) { String maxInstantTime = metaClient .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().requestedTime(); return maxInstantTime; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java index caeac4d322d3..746b39d451f6 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/LogCompactionExecutionHelper.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieWriteConfig; @@ -53,7 +52,7 @@ public class LogCompactionExecutionHelper execute() { extraMetadata.ifPresent(plan::setExtraMetadata); try { if (operationType.equals(WriteOperationType.COMPACT)) { - HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, + HoodieInstant compactionInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); table.getActiveTimeline().saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); } else { - HoodieInstant logCompactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, + HoodieInstant logCompactionInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.LOG_COMPACTION_ACTION, instantTime); table.getActiveTimeline().saveToLogCompactionRequested(logCompactionInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); @@ -133,7 +133,7 @@ private Option> getLatestDeltaCommitInfo() { if (deltaCommitsInfo.isPresent()) { return Option.of(Pair.of( deltaCommitsInfo.get().getLeft().countInstants(), - deltaCommitsInfo.get().getRight().getTimestamp())); + deltaCommitsInfo.get().getRight().requestedTime())); } return Option.empty(); } @@ -144,7 +144,7 @@ private Option> getLatestDeltaCommitInfoSinceLastCompactio if (deltaCommitsInfo.isPresent()) { return Option.of(Pair.of( deltaCommitsInfo.get().getLeft().countInstants(), - deltaCommitsInfo.get().getRight().getTimestamp())); + deltaCommitsInfo.get().getRight().requestedTime())); } return Option.empty(); } @@ -210,7 +210,7 @@ private boolean needCompact(CompactionTriggerStrategy compactionTriggerStrategy) } private Long parsedToSeconds(String time) { - return HoodieActiveTimeline.parseDateFromInstantTimeSafely(time).orElseThrow(() -> new HoodieCompactionException("Failed to parse timestamp " + time)) + return TimelineUtils.parseDateFromInstantTimeSafely(time).orElseThrow(() -> new HoodieCompactionException("Failed to parse timestamp " + time)) .getTime() / 1000; } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java index a84d6ad7b633..15c1aab12ace 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/compact/plan/generators/BaseHoodieCompactionPlanGenerator.java @@ -81,7 +81,7 @@ public HoodieCompactionPlan generateCompactionPlan(String compactionInstant) thr // TODO : check if maxMemory is not greater than JVM or executor memory // TODO - rollback any compactions in flight HoodieTableMetaClient metaClient = hoodieTable.getMetaClient(); - CompletionTimeQueryView completionTimeQueryView = new CompletionTimeQueryView(metaClient); + CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); List partitionPaths = FSUtils.getAllPartitionPaths( engineContext, metaClient.getStorage(), writeConfig.getMetadataConfig(), metaClient.getBasePath()); @@ -119,7 +119,7 @@ public HoodieCompactionPlan generateCompactionPlan(String compactionInstant) thr String lastCompletedInstantTime = hoodieTable.getMetaClient() .getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.ROLLBACK_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)) - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().requestedTime(); LOG.info("Last completed instant time " + lastCompletedInstantTime); Option instantRange = CompactHelpers.getInstance().getInstantRange(metaClient); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/AbstractIndexingCatchupTask.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/AbstractIndexingCatchupTask.java index 70be1b76f91b..1c4ccc35beec 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/AbstractIndexingCatchupTask.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/AbstractIndexingCatchupTask.java @@ -91,9 +91,9 @@ public void run() { } // if instant completed, ensure that there was metadata commit, else update metadata for this completed instant if (COMPLETED.equals(instant.getState())) { - String instantTime = instant.getTimestamp(); + String instantTime = instant.requestedTime(); Option metadataInstant = metadataMetaClient.reloadActiveTimeline() - .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); + .filterCompletedInstants().filter(i -> i.requestedTime().equals(instantTime)).firstInstant(); if (metadataInstant.isPresent()) { currentCaughtupInstant = instantTime; continue; @@ -110,17 +110,17 @@ public void run() { break; case CLEAN_ACTION: HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(metaClient, instant); - metadataWriter.update(cleanMetadata, instant.getTimestamp()); + metadataWriter.update(cleanMetadata, instant.requestedTime()); break; case RESTORE_ACTION: HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( metaClient.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(restoreMetadata, instant.getTimestamp()); + metadataWriter.update(restoreMetadata, instant.requestedTime()); break; case ROLLBACK_ACTION: HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( metaClient.getActiveTimeline().getInstantDetails(instant).get()); - metadataWriter.update(rollbackMetadata, instant.getTimestamp()); + metadataWriter.update(rollbackMetadata, instant.requestedTime()); break; default: throw new IllegalStateException("Unexpected value: " + instant.getAction()); @@ -150,8 +150,8 @@ public void run() { * @return null if instant is already caught up, else the instant after it is completed. */ HoodieInstant awaitInstantCaughtUp(HoodieInstant instant) { - if (!metadataCompletedInstants.isEmpty() && metadataCompletedInstants.contains(instant.getTimestamp())) { - currentCaughtupInstant = instant.getTimestamp(); + if (!metadataCompletedInstants.isEmpty() && metadataCompletedInstants.contains(instant.requestedTime())) { + currentCaughtupInstant = instant.requestedTime(); return null; } if (!instant.isCompleted()) { @@ -166,12 +166,12 @@ HoodieInstant awaitInstantCaughtUp(HoodieInstant instant) { } private void reloadTimelineWithWait(HoodieInstant instant) throws InterruptedException { - String instantTime = instant.getTimestamp(); + String instantTime = instant.requestedTime(); Option currentInstant; do { currentInstant = metaClient.reloadActiveTimeline() - .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); + .filterCompletedInstants().filter(i -> i.requestedTime().equals(instantTime)).firstInstant(); if (!currentInstant.isPresent() || !currentInstant.get().isCompleted()) { Thread.sleep(TIMELINE_RELOAD_INTERVAL_MILLIS); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RecordBasedIndexingCatchupTask.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RecordBasedIndexingCatchupTask.java index fc1c200ccd1f..da91af0c7709 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RecordBasedIndexingCatchupTask.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/RecordBasedIndexingCatchupTask.java @@ -58,10 +58,10 @@ public RecordBasedIndexingCatchupTask(HoodieTableMetadataWriter metadataWriter, @Override public void updateIndexForWriteAction(HoodieInstant instant) throws IOException { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); HoodieData records = readRecordKeysFromFileSlices(instant); - metadataWriter.update(commitMetadata, records, instant.getTimestamp()); + metadataWriter.update(commitMetadata, records, instant.requestedTime()); } private HoodieData readRecordKeysFromFileSlices(HoodieInstant instant) throws IOException { 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 fd45dbd898d8..8c46c498aa23 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 @@ -27,7 +27,6 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.HoodieTimer; @@ -63,10 +62,11 @@ import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS_INFLIGHT; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.HoodieTimeline.INDEXING_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.RESTORE_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; @@ -147,7 +147,7 @@ public Option execute() { String indexUptoInstant = indexPartitionInfos.get(0).getIndexUptoInstant(); LOG.info("Starting Index Building with base instant: " + indexUptoInstant); HoodieTimer timer = HoodieTimer.start(); - metadataWriter.buildMetadataPartitions(context, indexPartitionInfos, indexInstant.getTimestamp()); + metadataWriter.buildMetadataPartitions(context, indexPartitionInfos, indexInstant.requestedTime()); metrics.ifPresent(m -> m.updateMetrics(HoodieMetadataMetrics.INITIALIZE_STR, timer.endTimer())); // get remaining instants to catchup @@ -160,7 +160,7 @@ public Option execute() { .setConf(storageConf.newInstance()) .setBasePath(metadataBasePath).build(); Set metadataCompletedTimestamps = getCompletedArchivedAndActiveInstantsAfter(indexUptoInstant, metadataMetaClient).stream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + .map(HoodieInstant::requestedTime).collect(Collectors.toSet()); // index catchup for all remaining instants with a timeout currentCaughtupInstant = indexUptoInstant; @@ -225,7 +225,7 @@ private void abort(HoodieInstant indexInstant, Set requestedPartitions) }); // delete inflight instant - table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(HoodieTimeline.getIndexInflightInstant(indexInstant.getTimestamp())); + table.getMetaClient().reloadActiveTimeline().deleteInstantFileIfExists(instantGenerator.getIndexInflightInstant(indexInstant.requestedTime())); } private List getInstantsToCatchup(String indexUptoInstant) { @@ -240,7 +240,7 @@ private List getInstantsToCatchup(String indexUptoInstant) { // get all instants since the plan completed (both from active timeline and archived timeline) List instantsToIndex; if (catchupStartInstant.isPresent()) { - instantsToIndex = getRemainingArchivedAndActiveInstantsSince(catchupStartInstant.get().getTimestamp(), table.getMetaClient()); + instantsToIndex = getRemainingArchivedAndActiveInstantsSince(catchupStartInstant.get().requestedTime(), table.getMetaClient()); } else { instantsToIndex = getRemainingArchivedAndActiveInstantsSince(indexUptoInstant, table.getMetaClient()); } @@ -256,7 +256,7 @@ private HoodieInstant validateAndGetIndexInstant() { return table.getActiveTimeline() .filterPendingIndexTimeline() - .filter(instant -> instant.getTimestamp().equals(instantTime) && REQUESTED.equals(instant.getState())) + .filter(instant -> instant.requestedTime().equals(instantTime) && REQUESTED.equals(instant.getState())) .lastInstant() .orElseThrow(() -> new HoodieIndexException(String.format("No requested index instant found: %s", instantTime))); } @@ -269,7 +269,7 @@ private void updateTableConfigAndTimeline(HoodieInstant indexInstant, txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); updateMetadataPartitionsTableConfig(table.getMetaClient(), finalIndexPartitionInfos.stream().map(HoodieIndexPartitionInfo::getMetadataPartitionPath).collect(Collectors.toSet())); - table.getActiveTimeline().saveAsComplete(false, new HoodieInstant(true, INDEXING_ACTION, indexInstant.getTimestamp()), + table.getActiveTimeline().saveAsComplete(false, instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, INDEXING_ACTION, indexInstant.requestedTime()), TimelineMetadataUtils.serializeIndexCommitMetadata(indexCommitMetadata)); } finally { txnManager.endTransaction(Option.of(indexInstant)); @@ -298,11 +298,11 @@ private void catchupWithInflightWriters(HoodieTableMetadataWriter metadataWriter private static List getRemainingArchivedAndActiveInstantsSince(String instant, HoodieTableMetaClient metaClient) { List remainingInstantsToIndex = metaClient.getArchivedTimeline().getInstantsAsStream() - .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) + .filter(i -> compareTimestamps(i.requestedTime(), GREATER_THAN_OR_EQUALS, instant)) .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList()); remainingInstantsToIndex.addAll(metaClient.getActiveTimeline().findInstantsAfter(instant).getInstantsAsStream() - .filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), GREATER_THAN_OR_EQUALS, instant)) + .filter(i -> compareTimestamps(i.requestedTime(), GREATER_THAN_OR_EQUALS, instant)) .filter(i -> !INDEXING_ACTION.equals(i.getAction())) .collect(Collectors.toList())); return remainingInstantsToIndex; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java index dabbe9301c42..0d973ca0d418 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/ScheduleIndexActionExecutor.java @@ -24,7 +24,7 @@ import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -89,6 +89,7 @@ public Option execute() { validateBeforeScheduling(); // make sure that it is idempotent, check with previously pending index operations. Set indexesInflightOrCompleted = getInflightAndCompletedMetadataPartitions(table.getMetaClient().getTableConfig()); + InstantGenerator instantGenerator = table.getMetaClient().getInstantGenerator(); Set requestedPartitions = partitionIndexTypes.stream().map(MetadataPartitionType::getPartitionPath).collect(Collectors.toSet()); requestedPartitions.addAll(partitionPaths); @@ -103,7 +104,7 @@ public Option execute() { } List finalPartitionsToIndex = partitionIndexTypes.stream() .filter(p -> requestedPartitions.contains(p.getPartitionPath())).collect(Collectors.toList()); - final HoodieInstant indexInstant = HoodieTimeline.getIndexRequestedInstant(instantTime); + final HoodieInstant indexInstant = instantGenerator.getIndexRequestedInstant(instantTime); try { this.txnManager.beginTransaction(Option.of(indexInstant), Option.empty()); // get last completed instant @@ -133,7 +134,7 @@ public Option execute() { private HoodieIndexPartitionInfo buildIndexPartitionInfo(MetadataPartitionType partitionType, HoodieInstant indexUptoInstant) { // for functional index, we need to pass the index name as the partition name String partitionName = MetadataPartitionType.FUNCTIONAL_INDEX.equals(partitionType) ? config.getIndexingConfig().getIndexName() : partitionType.getPartitionPath(); - return new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, partitionName, indexUptoInstant.getTimestamp(), Collections.emptyMap()); + return new HoodieIndexPartitionInfo(LATEST_INDEX_PLAN_VERSION, partitionName, indexUptoInstant.requestedTime(), Collections.emptyMap()); } private void validateBeforeScheduling() { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/WriteStatBasedIndexingCatchupTask.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/WriteStatBasedIndexingCatchupTask.java index 7118f3ab4836..727028e1eb61 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/WriteStatBasedIndexingCatchupTask.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/index/WriteStatBasedIndexingCatchupTask.java @@ -48,8 +48,8 @@ public WriteStatBasedIndexingCatchupTask(HoodieTableMetadataWriter metadataWrite @Override public void updateIndexForWriteAction(HoodieInstant instant) throws IOException { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); - metadataWriter.updateFromWriteStatuses(commitMetadata, engineContext.emptyHoodieData(), instant.getTimestamp()); + metadataWriter.updateFromWriteStatuses(commitMetadata, engineContext.emptyHoodieData(), instant.requestedTime()); } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 30a02f2b458e..1d6fb2b544dc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -24,7 +24,6 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.client.transaction.TransactionManager; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -48,6 +47,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; + public abstract class BaseRestoreActionExecutor extends BaseActionExecutor { private static final Logger LOG = LoggerFactory.getLogger(BaseRestoreActionExecutor.class); @@ -71,7 +72,7 @@ public HoodieRestoreMetadata execute() { Option restoreInstant = table.getRestoreTimeline() .filterInflightsAndRequested() - .filter(instant -> instant.getTimestamp().equals(instantTime)) + .filter(instant -> instant.requestedTime().equals(instantTime)) .firstInstant(); if (!restoreInstant.isPresent()) { throw new HoodieRollbackException("No pending restore instants found to execute restore"); @@ -86,7 +87,7 @@ public HoodieRestoreMetadata execute() { } instantsToRollback.forEach(instant -> { - instantToMetadata.put(instant.getTimestamp(), Collections.singletonList(rollbackInstant(instant))); + instantToMetadata.put(instant.requestedTime(), Collections.singletonList(rollbackInstant(instant))); LOG.info("Deleted instant " + instant); }); @@ -106,7 +107,7 @@ private List getInstantsToRollback(HoodieInstant restoreInstant) // If restore crashed midway, there are chances that some commits are already rolled back, // but some are not. so, we can ignore those commits which are fully rolledback in previous attempt if any. Option rollbackInstantOpt = table.getActiveTimeline().getWriteTimeline() - .filter(instant -> instant.getTimestamp().equals(instantInfo.getCommitTime()) && instant.getAction().equals(instantInfo.getAction())).firstInstant(); + .filter(instant -> instant.requestedTime().equals(instantInfo.getCommitTime()) && instant.getAction().equals(instantInfo.getAction())).firstInstant(); if (rollbackInstantOpt.isPresent()) { instantsToRollback.add(rollbackInstantOpt.get()); } else { @@ -124,21 +125,21 @@ private HoodieRestoreMetadata finishRestore(Map instantsToRollback = table.getActiveTimeline().getRollbackTimeline() .getReverseOrderedInstants() - .filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), savepointToRestoreTimestamp)) + .filter(instant -> GREATER_THAN.test(instant.requestedTime(), savepointToRestoreTimestamp)) .collect(Collectors.toList()); instantsToRollback.forEach(entry -> { if (entry.isCompleted()) { table.getActiveTimeline().deleteCompletedRollback(entry); } - table.getActiveTimeline().deletePending(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, entry.getTimestamp())); - table.getActiveTimeline().deletePending(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, entry.getTimestamp())); + table.getActiveTimeline().deletePending(instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, entry.requestedTime())); + table.getActiveTimeline().deletePending(instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, entry.requestedTime())); }); LOG.info("Commits " + instantsRolledBack + " rollback is complete. Restored table to " + savepointToRestoreTimestamp); return restoreMetadata; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java index e71a8566f953..de0bebe6c5a5 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/restore/RestoreUtils.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.table.HoodieTable; @@ -42,7 +41,7 @@ public class RestoreUtils { */ public static HoodieRestorePlan getRestorePlan(HoodieTableMetaClient metaClient, HoodieInstant restoreInstant) throws IOException { - final HoodieInstant requested = HoodieTimeline.getRollbackRequestedInstant(restoreInstant); + final HoodieInstant requested = metaClient.getInstantGenerator().getRollbackRequestedInstant(restoreInstant); return TimelineMetadataUtils.deserializeAvroMetadata( metaClient.getActiveTimeline().readRestoreInfoAsBytes(requested).get(), HoodieRestorePlan.class); } @@ -52,7 +51,7 @@ public static String getSavepointToRestoreTimestampV1Schema(HoodieTable table, H String firstRollback = plan.getInstantsToRollback().get(plan.getInstantsToRollback().size() - 1).getCommitTime(); //find last instant before first rollback Option savepointInstance = table.getActiveTimeline().getSavePointTimeline().findInstantsBefore(firstRollback).lastInstant(); - return savepointInstance.isPresent() ? savepointInstance.get().getTimestamp() : null; + return savepointInstance.isPresent() ? savepointInstance.get().requestedTime() : null; } /** 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 e4094e868647..3246e207c395 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 @@ -29,6 +29,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; @@ -52,6 +53,9 @@ import java.util.Objects; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + public abstract class BaseRollbackActionExecutor extends BaseActionExecutor { private static final Logger LOG = LoggerFactory.getLogger(BaseRollbackActionExecutor.class); @@ -117,7 +121,7 @@ private HoodieRollbackMetadata runRollback(HoodieTable table, Hoodie finishRollback(inflightInstant, rollbackMetadata); // Finally, remove the markers post rollback. - WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.getTimestamp()) + WriteMarkersFactory.get(config.getMarkersType(), table, instantToRollback.requestedTime()) .quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); return rollbackMetadata; @@ -128,7 +132,7 @@ public HoodieRollbackMetadata execute() { table.getMetaClient().reloadActiveTimeline(); Option rollbackInstant = table.getRollbackTimeline() .filterInflightsAndRequested() - .filter(instant -> instant.getTimestamp().equals(instantTime)) + .filter(instant -> instant.requestedTime().equals(instantTime)) .firstInstant(); if (!rollbackInstant.isPresent()) { throw new HoodieRollbackException("No pending rollback instants found to execute rollback"); @@ -144,10 +148,10 @@ 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().getInstantsAsStream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .collect(Collectors.toList()); savepoints.forEach(s -> { - if (s.contains(instantToRollback.getTimestamp())) { + if (s.contains(instantToRollback.requestedTime())) { throw new HoodieRollbackException( "Could not rollback a savepointed commit. Delete savepoint first before rolling back" + s); } @@ -163,7 +167,7 @@ private void validateRollbackCommitSequence() { // Remove this once we support LAZY rollback of failed writes by default as parallel writing becomes the default // writer mode. if (config.getFailedWritesCleanPolicy().isEager() && !HoodieTableMetadata.isMetadataTable(config.getBasePath())) { - final String instantTimeToRollback = instantToRollback.getTimestamp(); + final String instantTimeToRollback = instantToRollback.requestedTime(); HoodieTimeline commitTimeline = table.getCompletedCommitsTimeline(); HoodieTimeline pendingCommitsTimeline = table.getPendingCommitsTimeline(); // Check validity of completed commit timeline. @@ -185,8 +189,8 @@ private void validateRollbackCommitSequence() { } List inflights = pendingCommitsTimeline.getInstantsAsStream() - .filter(instant -> !ClusteringUtils.isClusteringInstant(table.getActiveTimeline(), instant)) - .map(HoodieInstant::getTimestamp) + .filter(instant -> !ClusteringUtils.isClusteringInstant(table.getActiveTimeline(), instant, instantGenerator)) + .map(HoodieInstant::requestedTime) .collect(Collectors.toList()); if ((instantTimeToRollback != null) && !inflights.isEmpty() && (inflights.indexOf(instantTimeToRollback) != inflights.size() - 1)) { @@ -197,18 +201,20 @@ private void validateRollbackCommitSequence() { } private void rollBackIndex() { - if (!table.getIndex().rollbackCommit(instantToRollback.getTimestamp())) { + if (!table.getIndex().rollbackCommit(instantToRollback.requestedTime())) { throw new HoodieRollbackException("Rollback index changes failed, for time :" + instantToRollback); } LOG.info("Index rolled back for commits " + instantToRollback); } public List doRollbackAndGetStats(HoodieRollbackPlan hoodieRollbackPlan) { - final String instantTimeToRollback = instantToRollback.getTimestamp(); + final String instantTimeToRollback = instantToRollback.requestedTime(); final boolean isPendingCompaction = Objects.equals(HoodieTimeline.COMPACTION_ACTION, instantToRollback.getAction()) && !instantToRollback.isCompleted(); - final boolean isPendingClustering = !instantToRollback.isCompleted() && ClusteringUtils.isClusteringInstant(table.getMetaClient().getActiveTimeline(), instantToRollback); + final boolean isPendingClustering = !instantToRollback.isCompleted() + && ClusteringUtils.isClusteringInstant( + table.getMetaClient().getActiveTimeline(), instantToRollback, instantGenerator); validateSavepointRollbacks(); if (!isPendingCompaction && !isPendingClustering) { validateRollbackCommitSequence(); @@ -288,8 +294,8 @@ protected void deleteInflightAndRequestedInstant(boolean deleteInstant, activeTimeline.deletePending(instantToBeDeleted); if (instantToBeDeleted.isInflight() && !table.getMetaClient().getTimelineLayoutVersion().isNullVersion()) { // Delete corresponding requested instant - instantToBeDeleted = new HoodieInstant(HoodieInstant.State.REQUESTED, instantToBeDeleted.getAction(), - instantToBeDeleted.getTimestamp()); + instantToBeDeleted = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, instantToBeDeleted.getAction(), + instantToBeDeleted.requestedTime()); activeTimeline.deletePending(instantToBeDeleted); } LOG.info("Deleted pending commit " + instantToBeDeleted); @@ -299,7 +305,7 @@ protected void deleteInflightAndRequestedInstant(boolean deleteInstant, } protected void dropBootstrapIndexIfNeeded(HoodieInstant instantToRollback) { - if (HoodieTimeline.compareTimestamps(instantToRollback.getTimestamp(), HoodieTimeline.EQUALS, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)) { + if (compareTimestamps(instantToRollback.requestedTime(), EQUALS, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)) { LOG.info("Dropping bootstrap index as metadata bootstrap commit is getting rolled back !!"); BootstrapIndex.getBootstrapIndex(table.getMetaClient()).dropIndex(); } @@ -310,7 +316,6 @@ private void backupRollbackInstantsIfNeeded() { // Backup not required return; } - StoragePath backupDir = new StoragePath(config.getRollbackBackupDirectory()); if (!backupDir.isAbsolute()) { // Path specified is relative to the meta directory @@ -322,11 +327,11 @@ private void backupRollbackInstantsIfNeeded() { List instantsToBackup = new ArrayList<>(3); instantsToBackup.add(instantToRollback); if (instantToRollback.isCompleted()) { - instantsToBackup.add(HoodieTimeline.getInflightInstant(instantToRollback, table.getMetaClient())); - instantsToBackup.add(HoodieTimeline.getRequestedInstant(instantToRollback)); + instantsToBackup.add(TimelineUtils.getInflightInstant(instantToRollback, table.getMetaClient())); + instantsToBackup.add(instantGenerator.getRequestedInstant(instantToRollback)); } if (instantToRollback.isInflight()) { - instantsToBackup.add(HoodieTimeline.getRequestedInstant(instantToRollback)); + instantsToBackup.add(instantGenerator.getRequestedInstant(instantToRollback)); } for (HoodieInstant instant : instantsToBackup) { diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java index 9d2f727aeb90..0d674cfac4ce 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackHelper.java @@ -129,13 +129,13 @@ List> maybeDeleteAndCollectStats(HoodieEngineCo writer = HoodieLogFormat.newWriterBuilder() .onParentPath(FSUtils.constructAbsolutePath(metaClient.getBasePath(), rollbackRequest.getPartitionPath())) .withFileId(fileId) - .withDeltaCommit(instantToRollback.getTimestamp()) + .withDeltaCommit(instantToRollback.requestedTime()) .withStorage(metaClient.getStorage()) .withFileExtension(HoodieLogFile.DELTA_EXTENSION).build(); // generate metadata if (doDelete) { - Map header = generateHeader(instantToRollback.getTimestamp()); + Map header = generateHeader(instantToRollback.requestedTime()); // if update belongs to an existing log file // use the log file path from AppendResult in case the file handle may roll over filePath = writer.appendBlock(new HoodieCommandBlock(header)).logFile().getPath(); @@ -212,7 +212,7 @@ protected List deleteFiles(HoodieTableMetaClient metaClient, protected Map generateHeader(String commit) { // generate metadata Map header = new HashMap<>(3); - header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().getTimestamp()); + header.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, metaClient.getActiveTimeline().lastInstant().get().requestedTime()); header.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, commit); header.put(HoodieLogBlock.HeaderMetadataType.COMMAND_BLOCK_TYPE, String.valueOf(HoodieCommandBlock.HoodieCommandBlockTypeEnum.ROLLBACK_BLOCK.ordinal())); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java index 48af32751a37..9f688775307d 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackPlanActionExecutor.java @@ -104,16 +104,16 @@ private BaseRollbackPlanActionExecutor.RollbackStrategy getRollbackStrategy() { * @return Rollback Plan if generated */ protected Option requestRollback(String startRollbackTime) { - final HoodieInstant rollbackInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime); + final HoodieInstant rollbackInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, startRollbackTime); try { List rollbackRequests = new ArrayList<>(); if (!instantToRollback.isRequested()) { rollbackRequests.addAll(getRollbackStrategy().getRollbackRequests(instantToRollback)); } - HoodieRollbackPlan rollbackPlan = new HoodieRollbackPlan(new HoodieInstantInfo(instantToRollback.getTimestamp(), + HoodieRollbackPlan rollbackPlan = new HoodieRollbackPlan(new HoodieInstantInfo(instantToRollback.requestedTime(), instantToRollback.getAction()), rollbackRequests, LATEST_ROLLBACK_PLAN_VERSION); if (!skipTimelinePublish) { - if (table.getRollbackTimeline().filterInflightsAndRequested().containsInstant(rollbackInstant.getTimestamp())) { + if (table.getRollbackTimeline().filterInflightsAndRequested().containsInstant(rollbackInstant.requestedTime())) { LOG.warn("Request Rollback found with instant time " + rollbackInstant + ", hence skipping scheduling rollback"); } else { table.getActiveTimeline().saveToRollbackRequested(rollbackInstant, TimelineMetadataUtils.serializeRollbackPlan(rollbackPlan)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java index b21008676946..1d62a0911e94 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/ListingBasedRollbackStrategy.java @@ -51,6 +51,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.getHoodieCommitMetadata; import static org.apache.hudi.table.action.rollback.BaseRollbackHelper.EMPTY_STRING; @@ -147,11 +149,11 @@ public List getRollbackRequests(HoodieInstant instantToRo // and not corresponding base commit log files created with this as baseCommit since updates would // have been written to the log files. hoodieRollbackRequests.addAll(getHoodieRollbackRequests(partitionPath, - listBaseFilesToBeDeleted(instantToRollback.getTimestamp(), baseFileExtension, partitionPath, metaClient.getStorage()))); + listBaseFilesToBeDeleted(instantToRollback.requestedTime(), baseFileExtension, partitionPath, metaClient.getStorage()))); } else { // if this is part of a restore operation, we should rollback/delete entire file slice. hoodieRollbackRequests.addAll(getHoodieRollbackRequests(partitionPath, - listAllFilesSinceCommit(instantToRollback.getTimestamp(), baseFileExtension, partitionPath, + listAllFilesSinceCommit(instantToRollback.requestedTime(), baseFileExtension, partitionPath, metaClient))); } break; @@ -174,8 +176,8 @@ public List getRollbackRequests(HoodieInstant instantToRo return hoodieRollbackRequests.stream(); }, numPartitions); } catch (Exception e) { - LOG.error("Generating rollback requests failed for " + instantToRollback.getTimestamp(), e); - throw new HoodieRollbackException("Generating rollback requests failed for " + instantToRollback.getTimestamp(), e); + LOG.error("Generating rollback requests failed for " + instantToRollback.requestedTime(), e); + throw new HoodieRollbackException("Generating rollback requests failed for " + instantToRollback.requestedTime(), e); } } @@ -184,11 +186,11 @@ private List listAllFilesSinceCommit(String commit, String partitionPath, HoodieTableMetaClient metaClient) throws IOException { LOG.info("Collecting files to be cleaned/rolledback up for path " + partitionPath + " and commit " + commit); - CompletionTimeQueryView completionTimeQueryView = new CompletionTimeQueryView(metaClient); + CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); StoragePathFilter filter = (path) -> { if (path.toString().contains(baseFileExtension)) { String fileCommitTime = FSUtils.getCommitTime(path.getName()); - return HoodieTimeline.compareTimestamps(commit, HoodieTimeline.LESSER_THAN_OR_EQUALS, + return compareTimestamps(commit, LESSER_THAN_OR_EQUALS, fileCommitTime); } else if (FSUtils.isLogFile(path)) { String fileCommitTime = FSUtils.getDeltaCommitTimeFromLogPath(path); @@ -253,7 +255,7 @@ private List fetchFilesFromCommitMetadata(HoodieInstant instant String baseFileExtension, HoodieStorage storage) throws IOException { StoragePathFilter pathFilter = getPathFilter(baseFileExtension, - instantToRollback.getTimestamp()); + instantToRollback.requestedTime()); List filePaths = getFilesFromCommitMetadata(basePath, commitMetadata, partitionPath) .filter(entry -> { try { @@ -283,7 +285,7 @@ private List fetchFilesFromListFiles(HoodieInstant instantToRol String basePath, String baseFileExtension, HoodieStorage storage) throws IOException { - StoragePathFilter pathFilter = getPathFilter(baseFileExtension, instantToRollback.getTimestamp()); + StoragePathFilter pathFilter = getPathFilter(baseFileExtension, instantToRollback.requestedTime()); List filePaths = listFilesToBeDeleted(basePath, partitionPath); return storage.listDirectEntries(filePaths, pathFilter); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java index 8a63d6865ac4..ae7986e4803a 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/MarkerBasedRollbackStrategy.java @@ -65,7 +65,7 @@ public MarkerBasedRollbackStrategy(HoodieTable table, HoodieEngineCo public List getRollbackRequests(HoodieInstant instantToRollback) { try { List markerPaths = MarkerBasedRollbackUtils.getAllMarkerPaths( - table, context, instantToRollback.getTimestamp(), config.getRollbackParallelism()); + table, context, instantToRollback.requestedTime(), config.getRollbackParallelism()); int parallelism = Math.max(Math.min(markerPaths.size(), config.getRollbackParallelism()), 1); return context.map(markerPaths, markerFilePath -> { String typeStr = markerFilePath.substring(markerFilePath.lastIndexOf(".") + 1); @@ -77,11 +77,11 @@ public List getRollbackRequests(HoodieInstant instantToRo switch (type) { case MERGE: case CREATE: - return new HoodieRollbackRequest(partitionPath, fileId, instantToRollback.getTimestamp(), + return new HoodieRollbackRequest(partitionPath, fileId, instantToRollback.requestedTime(), Collections.singletonList(filePath.toString()), Collections.emptyMap()); case APPEND: - return new HoodieRollbackRequest(partitionPath, fileId, instantToRollback.getTimestamp(), Collections.emptyList(), + return new HoodieRollbackRequest(partitionPath, fileId, instantToRollback.requestedTime(), Collections.emptyList(), Collections.singletonMap(filePath.toString(), 1L)); default: throw new HoodieRollbackException("Unknown marker type, during rollback of " + instantToRollback); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java index d2983a9aa3ae..83558e592787 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RestorePlanActionExecutor.java @@ -22,7 +22,6 @@ import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRestorePlan; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; @@ -41,6 +40,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; + /** * Plans the restore action and add a restore.requested meta file to timeline. */ @@ -65,27 +66,27 @@ public RestorePlanActionExecutor(HoodieEngineContext context, @Override public Option execute() { - final HoodieInstant restoreInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.RESTORE_ACTION, instantTime); + final HoodieInstant restoreInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.RESTORE_ACTION, instantTime); try { // Get all the commits on the timeline after the provided commit time // rollback pending clustering instants first before other instants (See HUDI-3362) List pendingClusteringInstantsToRollback = table.getActiveTimeline().filterPendingReplaceOrClusteringTimeline() // filter only clustering related replacecommits (Not insert_overwrite related commits) - .filter(instant -> ClusteringUtils.isClusteringInstant(table.getActiveTimeline(), instant)) + .filter(instant -> ClusteringUtils.isClusteringInstant(table.getActiveTimeline(), instant, instantGenerator)) .getReverseOrderedInstants() - .filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), savepointToRestoreTimestamp)) + .filter(instant -> GREATER_THAN.test(instant.requestedTime(), savepointToRestoreTimestamp)) .collect(Collectors.toList()); // Get all the commits on the timeline after the provided commit time List commitInstantsToRollback = table.getActiveTimeline().getWriteTimeline() .getReverseOrderedInstants() - .filter(instant -> HoodieActiveTimeline.GREATER_THAN.test(instant.getTimestamp(), savepointToRestoreTimestamp)) + .filter(instant -> GREATER_THAN.test(instant.requestedTime(), savepointToRestoreTimestamp)) .filter(instant -> !pendingClusteringInstantsToRollback.contains(instant)) .collect(Collectors.toList()); // Combine both lists - first rollback pending clustering and then rollback all other commits List instantsToRollback = Stream.concat(pendingClusteringInstantsToRollback.stream(), commitInstantsToRollback.stream()) - .map(entry -> new HoodieInstantInfo(entry.getTimestamp(), entry.getAction())) + .map(entry -> new HoodieInstantInfo(entry.requestedTime(), entry.getAction())) .collect(Collectors.toList()); HoodieRestorePlan restorePlan = new HoodieRestorePlan(instantsToRollback, LATEST_RESTORE_PLAN_VERSION, savepointToRestoreTimestamp); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java index 0fe9d46df5db..8e2496fa619f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/rollback/RollbackUtils.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.table.log.block.HoodieCommandBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.storage.StoragePathInfo; @@ -55,7 +54,7 @@ public class RollbackUtils { public static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClient, HoodieInstant rollbackInstant) throws IOException { // TODO: add upgrade step if required. - final HoodieInstant requested = HoodieTimeline.getRollbackRequestedInstant(rollbackInstant); + final HoodieInstant requested = metaClient.getInstantGenerator().getRollbackRequestedInstant(rollbackInstant); return TimelineMetadataUtils.deserializeAvroMetadata( metaClient.getActiveTimeline().readRollbackInfoAsBytes(requested).get(), HoodieRollbackPlan.class); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java index b1ed3f2417a1..30ec3faf541e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointActionExecutor.java @@ -45,6 +45,8 @@ import static org.apache.hudi.client.utils.MetadataTableUtils.shouldUseBatchLookup; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeCleanerPlan; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeHoodieCleanMetadata; @@ -84,16 +86,16 @@ public HoodieSavepointMetadata execute() { } else { // clean is pending or inflight return deserializeCleanerPlan( - table.getActiveTimeline().getInstantDetails(new HoodieInstant(REQUESTED, instant.getAction(), instant.getTimestamp())).get()) + table.getActiveTimeline().getInstantDetails(instantGenerator.createNewInstant(REQUESTED, instant.getAction(), instant.requestedTime())).get()) .getEarliestInstantToRetain().getTimestamp(); } } catch (IOException e) { throw new HoodieSavepointException("Failed to savepoint " + instantTime, e); } - }).orElseGet(() -> table.getCompletedCommitsTimeline().firstInstant().get().getTimestamp()); + }).orElseGet(() -> table.getCompletedCommitsTimeline().firstInstant().get().requestedTime()); // Cannot allow savepoint time on a commit that could have been cleaned - ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN_OR_EQUALS, lastCommitRetained), + ValidationUtils.checkArgument(compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, lastCommitRetained), "Could not savepoint commit " + instantTime + " as this is beyond the lookup window " + lastCommitRetained); context.setJobStatus(this.getClass().getSimpleName(), "Collecting latest files for savepoint " + instantTime + " " + table.getConfig().getTableName()); @@ -142,9 +144,9 @@ public HoodieSavepointMetadata execute() { HoodieSavepointMetadata metadata = TimelineMetadataUtils.convertSavepointMetadata(user, comment, latestFilesMap); // Nothing to save in the savepoint table.getActiveTimeline().createNewInstant( - new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, instantTime)); + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, instantTime)); table.getActiveTimeline() - .saveAsComplete(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, instantTime), + .saveAsComplete(instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, instantTime), TimelineMetadataUtils.serializeSavepointMetadata(metadata)); LOG.info("Savepoint " + instantTime + " created"); return metadata; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java index b5e12ca5d6e6..149831f573cc 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/action/savepoint/SavepointHelpers.java @@ -33,7 +33,7 @@ public class SavepointHelpers { private static final Logger LOG = LoggerFactory.getLogger(SavepointHelpers.class); public static void deleteSavepoint(HoodieTable table, String savepointTime) { - HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + HoodieInstant savePoint = table.getMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { LOG.warn("No savepoint present " + savepointTime); @@ -41,7 +41,8 @@ public static void deleteSavepoint(HoodieTable table, String savepointTime) { } table.getActiveTimeline().revertToInflight(savePoint); - table.getActiveTimeline().deleteInflight(new HoodieInstant(true, HoodieTimeline.SAVEPOINT_ACTION, savepointTime)); + table.getActiveTimeline().deleteInflight(table.getMetaClient().createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, + savepointTime)); LOG.info("Savepoint " + savepointTime + " deleted"); } @@ -53,13 +54,13 @@ public static void validateSavepointRestore(HoodieTable table, String savepointT .filterCompletedAndCompactionInstants() .lastInstant(); ValidationUtils.checkArgument(lastInstant.isPresent()); - ValidationUtils.checkArgument(lastInstant.get().getTimestamp().equals(savepointTime), + ValidationUtils.checkArgument(lastInstant.get().requestedTime().equals(savepointTime), savepointTime + " is not the last commit after restoring to savepoint, last commit was " - + lastInstant.get().getTimestamp()); + + lastInstant.get().requestedTime()); } public static void validateSavepointPresence(HoodieTable table, String savepointTime) { - HoodieInstant savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); + HoodieInstant savePoint = table.getMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepointTime); boolean isSavepointPresent = table.getCompletedSavepointTimeline().containsInstant(savePoint); if (!isSavepointPresent) { throw new HoodieRollbackException("No savepoint for instantTime " + savepointTime); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java index d09cc74f14e9..72525aadfa73 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/marker/SimpleTransactionDirectMarkerBasedDetectionStrategy.java @@ -20,6 +20,8 @@ import org.apache.hudi.client.transaction.DirectMarkerTransactionManager; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieEarlyConflictDetectionException; import org.apache.hudi.storage.HoodieStorage; @@ -48,9 +50,10 @@ public SimpleTransactionDirectMarkerBasedDetectionStrategy( public void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDetectionException { DirectMarkerTransactionManager txnManager = new DirectMarkerTransactionManager((HoodieWriteConfig) config, storage, partitionPath, fileId); + InstantGenerator instantGenerator = TimelineLayout.fromVersion(activeTimeline.getTimelineLayoutVersion()).getInstantGenerator(); try { // Need to do transaction before create marker file when using early conflict detection - txnManager.beginTransaction(instantTime); + txnManager.beginTransaction(instantTime, instantGenerator); super.detectAndResolveConflictIfNecessary(); } catch (Exception e) { @@ -58,7 +61,7 @@ public void detectAndResolveConflictIfNecessary() throws HoodieEarlyConflictDete throw e; } finally { // End transaction after created marker file. - txnManager.endTransaction(instantTime); + txnManager.endTransaction(instantTime, instantGenerator); txnManager.close(); } } 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 f4eb9ef31cf1..b0016de2adfb 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 @@ -27,6 +27,7 @@ import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.storage.StoragePath; @@ -84,14 +85,15 @@ public static Option> getBaseAndLogFilePathsFromTimeline( HoodieTimeline timeline, HoodieInstant instant) throws IOException { if (!instant.isCompleted()) { throw new HoodieException("Cannot get base and log file paths from " - + "instant not completed: " + instant.getTimestamp()); + + "instant not completed: " + instant.requestedTime()); } switch (instant.getAction()) { case COMMIT_ACTION: case DELTA_COMMIT_ACTION: + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); final HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes( + layout.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); return Option.of(commitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream) .map(HoodieWriteStat::getPath).collect(Collectors.toSet())); @@ -120,7 +122,7 @@ 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) + if (!activeTimeline.filter(instant -> instant.requestedTime().equals(instantToRepair) && !instant.isCompleted()).empty()) { return Collections.emptyList(); } @@ -129,7 +131,7 @@ public static List findInstantFilesToRemove( boolean doesInstantExist = false; Option> filesFromTimeline = Option.empty(); Option instantOption = activeTimeline.filterCompletedInstants().filter( - instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant(); + instant -> instant.requestedTime().equals(instantToRepair)).firstInstant(); if (instantOption.isPresent()) { // Completed instant in active timeline doesInstantExist = true; @@ -137,7 +139,7 @@ public static List findInstantFilesToRemove( activeTimeline, instantOption.get()); } else { instantOption = archivedTimeline.filterCompletedInstants().filter( - instant -> instant.getTimestamp().equals(instantToRepair)).firstInstant(); + instant -> instant.requestedTime().equals(instantToRepair)).firstInstant(); if (instantOption.isPresent()) { // Completed instant in archived timeline doesInstantExist = true; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java index f4f176db1423..1286603ed015 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/EightToSevenDowngradeHandler.java @@ -21,11 +21,19 @@ import org.apache.hudi.common.config.ConfigProperty; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.versioning.v1.ActiveTimelineV1; +import org.apache.hudi.common.table.timeline.versioning.v1.CommitMetadataSerDeV1; +import org.apache.hudi.common.table.timeline.versioning.v2.CommitMetadataSerDeV2; +import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.keygen.constant.KeyGeneratorOptions; import org.apache.hudi.keygen.constant.KeyGeneratorType; @@ -44,6 +52,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.stream.Collectors; import static org.apache.hudi.common.table.HoodieTableConfig.TABLE_METADATA_PARTITIONS; @@ -72,20 +81,43 @@ public Map downgrade(HoodieWriteConfig config, HoodieEng HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getStorageConf().newInstance()).setBasePath(config.getBasePath()).build(); List instants = metaClient.getActiveTimeline().getInstants(); if (!instants.isEmpty()) { + InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); + CommitMetadataSerDeV2 commitMetadataSerDeV2 = new CommitMetadataSerDeV2(); + CommitMetadataSerDeV1 commitMetadataSerDeV1 = new CommitMetadataSerDeV1(); + ActiveTimelineV1 activeTimelineV1 = new ActiveTimelineV1(metaClient); + String tmpFilePrefix = "temp_commit_file_for_eight_to_seven_downgrade_"; context.map(instants, instant -> { - if (instant.getFileName().contains(UNDERSCORE)) { + String fileName = instantFileNameGenerator.getFileName(instant); + if (fileName.contains(UNDERSCORE)) { try { // Rename the metadata file name from the ${instant_time}_${completion_time}.action[.state] format in version 1.x to the ${instant_time}.action[.state] format in version 0.x. - StoragePath fromPath = new StoragePath(metaClient.getMetaPath(), instant.getFileName()); - StoragePath toPath = new StoragePath(metaClient.getMetaPath(), instant.getFileName().replaceAll(UNDERSCORE + "\\d+", "")); - boolean success = metaClient.getStorage().rename(fromPath, toPath); + StoragePath fromPath = new StoragePath(metaClient.getMetaPath(), fileName); + StoragePath toPath = new StoragePath(metaClient.getMetaPath(), fileName.replaceAll(UNDERSCORE + "\\d+", "")); + boolean success = true; + if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION) || instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) { + HoodieCommitMetadata commitMetadata = + commitMetadataSerDeV2.deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + Option data = commitMetadataSerDeV1.serialize(commitMetadata); + // Create a temporary file to store the json metadata. + String tmpFileName = tmpFilePrefix + UUID.randomUUID() + ".json"; + StoragePath tmpPath = new StoragePath(metaClient.getTempFolderPath(), tmpFileName); + String tmpPathStr = tmpPath.toUri().toString(); + activeTimelineV1.createFileInMetaPath(tmpPathStr, data, true); + // Note. this is a 2 step. First we create the V1 commit file and then delete file. If it fails in the middle, rerunning downgrade will be idempotent. + metaClient.getStorage().deleteFile(toPath); // First delete if it was created by previous failed downgrade. + success = metaClient.getStorage().rename(tmpPath, toPath); + metaClient.getStorage().deleteFile(fromPath); + } else { + success = metaClient.getStorage().rename(fromPath, toPath); + } // TODO: We need to rename the action-related part of the metadata file name here when we bring separate action name for clustering/compaction in 1.x as well. if (!success) { throw new HoodieIOException("an error that occurred while renaming " + fromPath + " to: " + toPath); } return true; } catch (IOException e) { - LOG.warn("Can not to complete the downgrade from version eight to version seven. The reason for failure is {}", e.getMessage()); + LOG.error("Can not to complete the downgrade from version eight to version seven. The reason for failure is {}", e.getMessage()); + throw new HoodieException(e); } } return false; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FiveToSixUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FiveToSixUpgradeHandler.java index da006f435b10..647700f5cc64 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FiveToSixUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/FiveToSixUpgradeHandler.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieUpgradeDowngradeException; import org.apache.hudi.storage.StoragePath; @@ -61,10 +62,12 @@ private void deleteCompactionRequestedFileFromAuxiliaryFolder(HoodieTable table) HoodieTableMetaClient metaClient = table.getMetaClient(); HoodieTimeline compactionTimeline = metaClient.getActiveTimeline().filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); + InstantFileNameGenerator factory = table.getMetaClient().getInstantFileNameGenerator(); + compactionTimeline.getInstantsAsStream().forEach( deleteInstant -> { LOG.info("Deleting instant " + deleteInstant + " in auxiliary meta path " + metaClient.getMetaAuxiliaryPath()); - StoragePath metaFile = new StoragePath(metaClient.getMetaAuxiliaryPath(), deleteInstant.getFileName()); + StoragePath metaFile = new StoragePath(metaClient.getMetaAuxiliaryPath(), factory.getFileName(deleteInstant)); try { if (metaClient.getStorage().exists(metaFile)) { metaClient.getStorage().deleteFile(metaFile); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java index f8bda1e21a48..9e4a1fbd6380 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/OneToZeroDowngradeHandler.java @@ -47,7 +47,7 @@ public Map downgrade( List commits = inflightTimeline.getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant inflightInstant : commits) { // delete existing markers - WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, inflightInstant.getTimestamp()); + WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, inflightInstant.requestedTime()); writeMarkers.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism()); } return Collections.emptyMap(); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java index 10cb8dba841d..59193bfc9ceb 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/TwoToOneDowngradeHandler.java @@ -65,7 +65,7 @@ public Map downgrade( // Converts the markers in new format to old format of direct markers try { convertToDirectMarkers( - inflightInstant.getTimestamp(), table, context, config.getMarkersDeleteParallelism()); + inflightInstant.requestedTime(), table, context, config.getMarkersDeleteParallelism()); } catch (IOException e) { throw new HoodieException("Converting marker files to DIRECT style failed during downgrade", e); } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java index 1b5cbe591387..daedf13cacbe 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/UpgradeDowngradeUtils.java @@ -25,10 +25,11 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieCompactionConfig; @@ -76,10 +77,12 @@ public static void runCompaction(HoodieTable table, HoodieEngineContext context, */ public static void syncCompactionRequestedFileToAuxiliaryFolder(HoodieTable table) { HoodieTableMetaClient metaClient = table.getMetaClient(); - HoodieTimeline compactionTimeline = new HoodieActiveTimeline(metaClient, false).filterPendingCompactionTimeline() + TimelineFactory timelineFactory = metaClient.getTimelineLayout().getTimelineFactory(); + InstantFileNameGenerator instantFileNameGenerator = metaClient.getInstantFileNameGenerator(); + HoodieTimeline compactionTimeline = timelineFactory.createActiveTimeline(metaClient, false).filterPendingCompactionTimeline() .filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED); compactionTimeline.getInstantsAsStream().forEach(instant -> { - String fileName = instant.getFileName(); + String fileName = instantFileNameGenerator.getFileName(instant); try { if (!metaClient.getStorage().exists(new StoragePath(metaClient.getMetaAuxiliaryPath(), fileName))) { FileIOUtils.copy(metaClient.getStorage(), 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 37b3660d57ab..3d261a8eb70b 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 @@ -25,7 +25,6 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.marker.MarkerType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; @@ -44,6 +43,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.EQUALS; + /** * Upgrade handle to assist in upgrading hoodie table from version 0 to 1. */ @@ -56,7 +57,7 @@ public Map upgrade( // fetch pending commit info HoodieTable table = upgradeDowngradeHelper.getTable(config, context); HoodieTimeline inflightTimeline = table.getMetaClient().getCommitsTimeline().filterPendingExcludingCompactionAndLogCompaction(); - List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::getTimestamp) + List commits = inflightTimeline.getReverseOrderedInstants().map(HoodieInstant::requestedTime) .collect(Collectors.toList()); if (!commits.isEmpty() && instantTime != null) { // ignore the latest inflight commit since a new commit would have been started, and we need to fix any pending commits from previous launch @@ -87,7 +88,7 @@ protected void recreateMarkers(final String commitInstantTime, try { // fetch hoodie instant Option commitInstantOpt = Option.fromJavaOptional(table.getActiveTimeline().getCommitsTimeline().getInstantsAsStream() - .filter(instant -> HoodieActiveTimeline.EQUALS.test(instant.getTimestamp(), commitInstantTime)) + .filter(instant -> EQUALS.test(instant.requestedTime(), commitInstantTime)) .findFirst()); if (commitInstantOpt.isPresent()) { // delete existing markers @@ -116,7 +117,7 @@ protected void recreateMarkers(final String commitInstantTime, List getListBasedRollBackStats(HoodieTable table, HoodieEngineContext context, Option commitInstantOpt) { List hoodieRollbackRequests = - new ListingBasedRollbackStrategy(table, context, table.getConfig(), commitInstantOpt.get().getTimestamp(), false) + new ListingBasedRollbackStrategy(table, context, table.getConfig(), commitInstantOpt.get().requestedTime(), false) .getRollbackRequests(commitInstantOpt.get()); return new BaseRollbackHelper(table.getMetaClient(), table.getConfig()) .collectRollbackStats(context, commitInstantOpt.get(), hoodieRollbackRequests); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java index 66070ee818b8..1cc8c5f4e7fa 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java @@ -25,6 +25,8 @@ import java.util.Collections; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; + /** * Instant triple for testing. */ @@ -35,8 +37,8 @@ public class DummyActiveAction extends ActiveAction { * Only for testing purpose. */ public DummyActiveAction(HoodieInstant completed, byte[] commitMetadata) { - super(new HoodieInstant(HoodieInstant.State.REQUESTED, completed.getAction(), completed.getTimestamp()), - new HoodieInstant(HoodieInstant.State.INFLIGHT, completed.getAction(), completed.getTimestamp()), + super(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, completed.getAction(), completed.requestedTime()), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, completed.getAction(), completed.requestedTime()), Collections.singletonList(completed)); this.commitMetadata = commitMetadata; } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java index 631aeebd58e3..10ad2750e474 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/HoodieTestCommitGenerator.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.InProcessTimeGenerator; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -56,6 +55,8 @@ import static org.apache.hudi.common.table.log.HoodieLogFormat.DEFAULT_WRITE_TOKEN; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; public class HoodieTestCommitGenerator { public static final String BASE_FILE_WRITE_TOKEN = "1-0-1"; @@ -124,10 +125,10 @@ public static String getLogFilename(String instantTime, String fileId) { public static void createCommitAndDataFiles( String basePath, String instantTime, Map>> partitionPathToFileIdAndNameMap) throws IOException { - String commitFilename = HoodieTimeline.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()); + String commitFilename = INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()); HoodieCommitMetadata commitMetadata = generateCommitMetadata(partitionPathToFileIdAndNameMap, Collections.emptyMap()); - createCommitFileWithMetadata(basePath, new HadoopStorageConfiguration(true), commitFilename, serializeCommitMetadata(commitMetadata).get()); + createCommitFileWithMetadata(basePath, new HadoopStorageConfiguration(true), commitFilename, serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); for (String partitionPath : partitionPathToFileIdAndNameMap.keySet()) { createPartitionMetaFile(basePath, partitionPath); partitionPathToFileIdAndNameMap.get(partitionPath) diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java index 61ad8aa60019..23e72a24edbf 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/timeline/TestCompletionTimeQueryView.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.timeline; import org.apache.hudi.DummyActiveAction; +import org.apache.hudi.client.timeline.versioning.v2.LSMTimelineWriter; import org.apache.hudi.common.engine.TaskContextSupplier; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; @@ -26,8 +27,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.ActiveAction; import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; @@ -48,6 +49,9 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -69,7 +73,8 @@ void testReadCompletionTime() throws Exception { HoodieTableMetaClient metaClient = HoodieTestUtils.init( HoodieTestUtils.getDefaultStorageConf(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); - try (CompletionTimeQueryView view = new CompletionTimeQueryView(metaClient, String.format("%08d", 3))) { + try (CompletionTimeQueryView view = + metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { // query completion time from LSM timeline for (int i = 3; i < 7; i++) { assertThat(view.getCompletionTime(String.format("%08d", i)).orElse(""), is(String.format("%08d", i + 1000))); @@ -99,7 +104,8 @@ void testReadStartTime() throws Exception { HoodieTableMetaClient metaClient = HoodieTestUtils.init( HoodieTestUtils.getDefaultStorageConf(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); prepareTimeline(tablePath, metaClient); - try (CompletionTimeQueryView view = new CompletionTimeQueryView(metaClient, String.format("%08d", 3))) { + try (CompletionTimeQueryView view = + metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient, String.format("%08d", 3))) { // query start time from LSM timeline assertThat(getInstantTimeSetFormattedString(view, 3 + 1000, 6 + 1000), is("00000003,00000004,00000005,00000006")); // query start time from active timeline @@ -134,11 +140,11 @@ private void prepareTimeline(String tablePath, HoodieTableMetaClient metaClient) testTable.addCommit(instantTime, Option.of(completionTime), Option.of(metadata)); activeActions.add( new DummyActiveAction( - new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", instantTime, completionTime), - serializeCommitMetadata(metadata).get())); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", instantTime, completionTime), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata).get())); } testTable.addRequestedCommit(String.format("%08d", 11)); - List instants = new HoodieActiveTimeline(metaClient, false).getInstantsAsStream().sorted().collect(Collectors.toList()); + List instants = TIMELINE_FACTORY.createActiveTimeline(metaClient, false).getInstantsAsStream().sorted().collect(Collectors.toList()); LSMTimelineWriter writer = LSMTimelineWriter.getInstance(writeConfig, getMockHoodieTable(metaClient)); // archive [1,2], [3,4], [5,6] separately writer.write(activeActions.subList(0, 2), Option.empty(), Option.empty()); @@ -146,8 +152,8 @@ private void prepareTimeline(String tablePath, HoodieTableMetaClient metaClient) writer.write(activeActions.subList(4, 6), Option.empty(), Option.empty()); // reconcile the active timeline instants.subList(0, 3 * 6).forEach( - instant -> HoodieActiveTimeline.deleteInstantFile(metaClient.getStorage(), - metaClient.getMetaPath(), instant)); + instant -> TimelineUtils.deleteInstantFile(metaClient.getStorage(), + metaClient.getMetaPath(), instant, INSTANT_FILE_NAME_GENERATOR)); ValidationUtils.checkState( metaClient.reloadActiveTimeline().filterCompletedInstants().countInstants() == 4, "should archive 6 instants with 4 as active"); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java index 3150575bca5d..7be23b9c1469 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestBucketIndexConcurrentFileWritesConflictResolutionStrategy.java @@ -40,6 +40,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; + public class TestBucketIndexConcurrentFileWritesConflictResolutionStrategy extends HoodieCommonTestHarness { @BeforeEach @@ -55,7 +57,7 @@ public void testNoConcurrentWrites() throws Exception { Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); @@ -73,7 +75,7 @@ public void testConcurrentWrites() throws Exception { createInflightCommit(HoodieTestTable.makeNewCommitTime(), HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); Assertions.assertEquals(0, candidateInstants.count()); @@ -92,7 +94,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except String newInstantTime = metaClient.createNewInstantTime(); createCommit(newInstantTime); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH); metaClient.reloadActiveTimeline(); @@ -124,7 +126,7 @@ public void testConcurrentWritesWithDifferentPartition() throws Exception { String newInstantTime = metaClient.createNewInstantTime(); createCommit(newInstantTime); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new BucketIndexConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH); metaClient.reloadActiveTimeline(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java index 1ebe3f05d1e0..f9feceee3eb6 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java @@ -40,6 +40,8 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; + public class TestConflictResolutionStrategyUtil { public static void createCommit(String instantTime, HoodieTableMetaClient metaClient) throws Exception { @@ -250,7 +252,7 @@ public static void createCompleteReplace(String instantTime, WriteOperationType writeStat.setFileId("file-2"); replaceMetadata.addWriteStat(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, writeStat); replaceMetadata.setOperationType(writeOperationType); - FileCreateUtils.createReplaceCommit(metaClient.getBasePath().toString(), instantTime, replaceMetadata); + FileCreateUtils.createReplaceCommit(COMMIT_METADATA_SER_DE, metaClient.getBasePath().toString(), instantTime, replaceMetadata); } public static void createPendingCompaction(String instantTime, HoodieTableMetaClient metaClient) throws Exception { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestPreferWriterConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestPreferWriterConflictResolutionStrategy.java index 4b1440904a11..4d68ad8d6b56 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestPreferWriterConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestPreferWriterConflictResolutionStrategy.java @@ -44,6 +44,7 @@ import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createReplace; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createClusterInflight; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createClusterRequested; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; public class TestPreferWriterConflictResolutionStrategy extends HoodieCommonTestHarness { @@ -65,7 +66,7 @@ public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exc String newInstantTime = metaClient.createNewInstantTime(); createCompactionRequested(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( Collectors.toList()); @@ -90,14 +91,14 @@ public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Ex Thread.sleep(1000); createCompaction(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( Collectors.toList()); // writer 1 conflicts with compaction 1 Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(newInstantTime, candidateInstants.get(0).getTimestamp()); + Assertions.assertEquals(newInstantTime, candidateInstants.get(0).requestedTime()); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); Assertions.assertTrue(strategy.hasConflict(thisCommitOperation, thatCommitOperation)); @@ -122,7 +123,7 @@ public void testConcurrentWritesWithInterleavingCompaction() throws Exception { String newInstantTime = metaClient.createNewInstantTime(); createCompactionRequested(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newInstantTime)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newInstantTime)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); // TODO Create method to create compactCommitMetadata // HoodieCommitMetadata currentMetadata = createCommitMetadata(newInstantTime); @@ -130,7 +131,7 @@ public void testConcurrentWritesWithInterleavingCompaction() throws Exception { Collectors.toList()); // writer 1 conflicts with compaction 1 Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(currentWriterInstant, candidateInstants.get(0).getTimestamp()); + Assertions.assertEquals(currentWriterInstant, candidateInstants.get(0).requestedTime()); // TODO: Once compactCommitMetadata is created use that to verify resolveConflict method. } @@ -150,7 +151,7 @@ public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception String currentWriterInstant = metaClient.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( Collectors.toList()); @@ -175,7 +176,7 @@ public void testConcurrentWritesWithInterleavingScheduledCluster() throws Except createClusterRequested(newInstantTime, metaClient); createClusterInflight(newInstantTime, WriteOperationType.CLUSTER, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( Collectors.toList()); @@ -205,14 +206,14 @@ public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Excep String replaceWriterInstant = metaClient.createNewInstantTime(); createCluster(replaceWriterInstant, WriteOperationType.CLUSTER, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); metaClient.reloadActiveTimeline(); List candidateInstants = strategy .getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant) .collect(Collectors.toList()); Assertions.assertEquals(1, candidateInstants.size()); - Assertions.assertEquals(replaceWriterInstant, candidateInstants.get(0).getTimestamp()); + Assertions.assertEquals(replaceWriterInstant, candidateInstants.get(0).requestedTime()); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); ConcurrentOperation thatCommitOperation = new ConcurrentOperation(candidateInstants.get(0), metaClient); ConcurrentOperation thisCommitOperation = new ConcurrentOperation(currentInstant.get(), currentMetadata); @@ -241,7 +242,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Excep String newInstantTime = metaClient.createNewInstantTime(); createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); - Option currentInstant = Option.of(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); PreferWriterConflictResolutionStrategy strategy = new PreferWriterConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); List candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant).collect( diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java index a2ace63bdb60..ee08b94da95f 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategy.java @@ -40,6 +40,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCluster; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createClusterRequested; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommit; @@ -70,7 +71,7 @@ public void testNoConcurrentWrites() throws Exception { Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); @@ -88,7 +89,7 @@ public void testConcurrentWrites() throws Exception { createInflightCommit(HoodieTestTable.makeNewCommitTime(), metaClient); Option lastSuccessfulInstant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant(); newInstantTime = HoodieTestTable.makeNewCommitTime(); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, newInstantTime)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); Stream candidateInstants = strategy.getCandidateInstants(metaClient, currentInstant.get(), lastSuccessfulInstant); Assertions.assertTrue(candidateInstants.count() == 0); @@ -107,7 +108,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulCommit() throws Except String newInstantTime = metaClient.createNewInstantTime(); createCommit(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -135,7 +136,7 @@ public void testConcurrentWritesWithReplaceInflightCommit() throws Exception { // writer 1 starts String currentWriterInstant = metaClient.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); // writer 2 starts and finishes String newInstantTime = metaClient.createNewInstantTime(); @@ -174,7 +175,7 @@ public void testConcurrentWritesWithInterleavingScheduledCompaction() throws Exc String newInstantTime = metaClient.createNewInstantTime(); createCompactionRequested(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -206,7 +207,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulCompaction() throws Ex String newInstantTime = metaClient.createNewInstantTime(); createCompaction(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -241,7 +242,7 @@ public void testConcurrentWriteAndCompactionScheduledEarlier() throws Exception String currentWriterInstant = metaClient.createNewInstantTime(); createInflightCommit(currentWriterInstant, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -264,7 +265,7 @@ public void testConcurrentWritesWithInterleavingScheduledCluster() throws Except String newInstantTime = metaClient.createNewInstantTime(); createClusterRequested(newInstantTime, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -296,7 +297,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulCluster() throws Excep String newInstantTime = metaClient.createNewInstantTime(); createCluster(newInstantTime, WriteOperationType.CLUSTER, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -328,7 +329,7 @@ public void testConcurrentWritesWithInterleavingSuccessfulReplace() throws Excep String newInstantTime = metaClient.createNewInstantTime(); createReplace(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -362,7 +363,7 @@ public void tstConcurrentWritesWithPendingInsertOverwriteReplace() throws Except String newInstantTime = metaClient.createNewInstantTime(); createPendingInsertOverwrite(newInstantTime, WriteOperationType.INSERT_OVERWRITE, metaClient); - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); @@ -406,7 +407,7 @@ public void testConcurrentWritesWithPendingInstants() throws Exception { createCompleteCommit(commitC4, metaClient); // step6: do check - Option currentInstant = Option.of(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); + Option currentInstant = Option.of(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, currentWriterInstant)); SimpleConcurrentFileWritesConflictResolutionStrategy strategy = new SimpleConcurrentFileWritesConflictResolutionStrategy(); // make sure c3 has conflict with C1,C11,C12,C4; HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant, "file-2"); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategyWithMORTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategyWithMORTable.java index 8f5311b14c0a..c94d3009cd90 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategyWithMORTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestSimpleConcurrentFileWritesConflictResolutionStrategyWithMORTable.java @@ -40,6 +40,7 @@ import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createCommitMetadata; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createInflightCommit; import static org.apache.hudi.client.transaction.TestConflictResolutionStrategyUtil.createPendingCompaction; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; public class TestSimpleConcurrentFileWritesConflictResolutionStrategyWithMORTable extends HoodieCommonTestHarness { @Override @@ -69,7 +70,7 @@ public void testConcurrentWritesWithInterleavingInflightCompaction() throws Exce // Writer 1 tries to commit. Option currentInstant = Option.of( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, currentWriterInstant)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, currentWriterInstant)); HoodieCommitMetadata currentMetadata = createCommitMetadata(currentWriterInstant); metaClient.reloadActiveTimeline(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java index bf11ace20ced..a14870421191 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestTransactionManager.java @@ -44,6 +44,7 @@ import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -266,6 +267,6 @@ public void testTransactionsWithUncheckedLockProviderRuntimeException() { } private Option getInstant(String timestamp) { - return Option.of(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, timestamp)); + return Option.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, timestamp)); } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java index 3a1632737efa..e55d9e614f62 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestDeletePartitionUtils.java @@ -37,6 +37,7 @@ import java.util.HashMap; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -99,7 +100,7 @@ private static CompactionOperation getCompactionOperation(boolean hasPendingJobI private static Stream> createFileGroupsInPendingClustering(boolean hasFileGroupsInPendingClustering) { HoodieFileGroupId hoodieFileGroupId = new HoodieFileGroupId(getPartitionName(hasFileGroupsInPendingClustering), "fileId"); - HoodieInstant hoodieInstant = new HoodieInstant(true, "replacecommit", HARDCODED_INSTANT_TIME); + HoodieInstant hoodieInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, "replacecommit", HARDCODED_INSTANT_TIME); return Stream.of(Pair.of(hoodieFileGroupId, hoodieInstant)); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java index 83d4da9dd549..53839cabddb3 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.ActiveAction; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.MetadataConversionUtils; import org.apache.hudi.common.testutils.HoodieTestTable; @@ -57,6 +56,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -93,7 +94,7 @@ private void prepareLegacyArchivedTimeline(HoodieTableMetaClient metaClient) thr HoodieCommitMetadata metadata = testTable.createCommitMetadata(instantTime, WriteOperationType.INSERT, Arrays.asList("par1", "par2"), 10, false); testTable.addCommit(instantTime, Option.of(metadata)); } - List instants = new HoodieActiveTimeline(metaClient, false).getInstantsAsStream().sorted().collect(Collectors.toList()); + List instants = TIMELINE_FACTORY.createActiveTimeline(metaClient, false).getInstantsAsStream().sorted().collect(Collectors.toList()); // archive 2 times to have 2 log files. archive(metaClient, instants.subList(0, instants.size() / 2)); archive(metaClient, instants.subList(instants.size() / 2, instants.size())); @@ -119,7 +120,7 @@ public void archive(HoodieTableMetaClient metaClient, List instan try { records.add(convertToAvroRecord(hoodieInstant, metaClient)); } catch (Exception e) { - LOG.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e); + LOG.error("Failed to archive commits, .commit file: " + INSTANT_FILE_NAME_GENERATOR.getFileName(hoodieInstant), e); throw e; } } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieArchivedTimeline.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieArchivedTimeline.java index 1d489e611fc8..833115c2af66 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieArchivedTimeline.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieArchivedTimeline.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.table.timeline; import org.apache.hudi.DummyActiveAction; -import org.apache.hudi.client.timeline.LSMTimelineWriter; +import org.apache.hudi.client.timeline.versioning.v2.LSMTimelineWriter; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.engine.HoodieLocalEngineContext; import org.apache.hudi.common.engine.LocalTaskContextSupplier; @@ -40,6 +40,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -64,13 +65,13 @@ public void testLoadingInstantsIncrementally() throws Exception { writeArchivedTimeline(10, 10000000); // now we got 500 instants spread in 5 parquets. HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline("10000043"); - assertThat(archivedTimeline.firstInstant().map(HoodieInstant::getTimestamp).orElse(""), is("10000043")); - assertThat(archivedTimeline.lastInstant().map(HoodieInstant::getTimestamp).orElse(""), is("10000050")); + assertThat(archivedTimeline.firstInstant().map(HoodieInstant::requestedTime).orElse(""), is("10000043")); + assertThat(archivedTimeline.lastInstant().map(HoodieInstant::requestedTime).orElse(""), is("10000050")); // load incrementally archivedTimeline.reload("10000034"); - assertThat(archivedTimeline.firstInstant().map(HoodieInstant::getTimestamp).orElse(""), is("10000034")); + assertThat(archivedTimeline.firstInstant().map(HoodieInstant::requestedTime).orElse(""), is("10000034")); archivedTimeline.reload("10000011"); - assertThat(archivedTimeline.firstInstant().map(HoodieInstant::getTimestamp).orElse(""), is("10000011")); + assertThat(archivedTimeline.firstInstant().map(HoodieInstant::requestedTime).orElse(""), is("10000011")); } // ------------------------------------------------------------------------- @@ -90,9 +91,9 @@ private void writeArchivedTimeline(int batchSize, long startTs) throws Exception long instantTimeTs = startTs + i; String instantTime = String.valueOf(instantTimeTs); String completionTime = String.valueOf(instantTimeTs + 10); - HoodieInstant instant = new HoodieInstant(HoodieInstant.State.COMPLETED, "commit", instantTime, completionTime); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", instantTime, completionTime); HoodieCommitMetadata metadata = testTable.createCommitMetadata(instantTime, WriteOperationType.INSERT, Arrays.asList("par1", "par2"), 10, false); - byte[] serializedMetadata = TimelineMetadataUtils.serializeCommitMetadata(metadata).get(); + byte[] serializedMetadata = TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata).get(); instantBuffer.add(new DummyActiveAction(instant, serializedMetadata)); if (i % batchSize == 0) { // archive 10 instants each time diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java index 9f51cb41dc92..b4a2f5a2d46c 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/common/testutils/HoodieMetadataTestTable.java @@ -41,6 +41,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createDeltaCommit; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; /** * {@link HoodieTestTable} impl used for testing metadata. This class does synchronous updates to HoodieTableMetadataWriter if non null. @@ -96,9 +97,9 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy // DT should be committed after MDT. if (!createInflightCommit) { if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { - createCommit(basePath, commitTime, Option.of(commitMetadata)); + createCommit(COMMIT_METADATA_SER_DE, basePath, commitTime, Option.of(commitMetadata)); } else { - createDeltaCommit(basePath, commitTime, commitMetadata); + createDeltaCommit(COMMIT_METADATA_SER_DE, basePath, commitTime, commitMetadata); } this.inflightCommits().remove(commitTime); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/TestCleanPlanner.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/TestCleanPlanner.java index 2d8b7e8354cb..405df6aeaa0d 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/TestCleanPlanner.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/TestCleanPlanner.java @@ -37,10 +37,11 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.versioning.v2.BaseTimelineV2; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.Option; @@ -73,6 +74,10 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_PARSER; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.apache.hudi.common.util.CleanerUtils.CLEAN_METADATA_VERSION_2; import static org.apache.hudi.common.util.CleanerUtils.SAVEPOINTED_TIMESTAMPS; @@ -99,6 +104,9 @@ void setUp() { when(sliceView.getPendingLogCompactionOperations()).thenReturn(Stream.empty()); HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); when(mockHoodieTable.getMetaClient()).thenReturn(metaClient); + TimelineLayout layout = mock(TimelineLayout.class); + when(metaClient.getTimelineLayout()).thenReturn(layout); + when(layout.getCommitMetadataSerDe()).thenReturn(COMMIT_METADATA_SER_DE); HoodieTableConfig tableConfig = new HoodieTableConfig(); when(metaClient.getTableConfig()).thenReturn(tableConfig); HoodieTimeline mockCompletedCommitsTimeline = mock(HoodieTimeline.class); @@ -112,7 +120,12 @@ void testGetDeletePaths(HoodieWriteConfig config, String earliestInstant, List replacedFileGroups, Pair> expected) throws IOException { SyncableFileSystemView mockFsView = mock(SyncableFileSystemView.class); + HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); + when(mockHoodieTable.getMetaClient()).thenReturn(mockMetaClient); when(mockHoodieTable.getHoodieView()).thenReturn(mockFsView); + when(mockHoodieTable.getInstantGenerator()).thenReturn(INSTANT_GENERATOR); + when(mockHoodieTable.getInstantFileNameGenerator()).thenReturn(INSTANT_FILE_NAME_GENERATOR); + when(mockHoodieTable.getInstantFileNameParser()).thenReturn(INSTANT_FILE_NAME_PARSER); // setup savepoint mocks Set savepointTimestamps = savepoints.stream().map(Pair::getLeft).collect(Collectors.toSet()); @@ -121,8 +134,9 @@ void testGetDeletePaths(HoodieWriteConfig config, String earliestInstant, List> savepoint : savepoints) { - HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, savepoint.getLeft()); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepoint.getLeft()); when(activeTimeline.getInstantDetails(instant)).thenReturn(savepoint.getRight()); + when(mockMetaClient.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepoint.getLeft())).thenReturn(instant); } } String partitionPath = "partition1"; @@ -137,7 +151,7 @@ void testGetDeletePaths(HoodieWriteConfig config, String earliestInstant, List cleanPlanner = new CleanPlanner<>(context, mockHoodieTable, config); - HoodieInstant earliestCommitToRetain = new HoodieInstant(COMPLETED, "COMMIT", earliestInstant); + HoodieInstant earliestCommitToRetain = INSTANT_GENERATOR.createNewInstant(COMPLETED, "COMMIT", earliestInstant); Pair> actual = cleanPlanner.getDeletePaths(partitionPath, Option.of(earliestCommitToRetain)); assertEquals(expected, actual); } @@ -159,10 +173,13 @@ void testPartitionsForIncrCleaning(boolean isPartitioned, HoodieWriteConfig conf // setup savepoint mocks Set savepointTimestamps = savepoints.keySet().stream().collect(Collectors.toSet()); when(mockHoodieTable.getSavepointTimestamps()).thenReturn(savepointTimestamps); + when(mockHoodieTable.getInstantGenerator()).thenReturn(INSTANT_GENERATOR); + when(mockHoodieTable.getInstantFileNameGenerator()).thenReturn(INSTANT_FILE_NAME_GENERATOR); + when(mockHoodieTable.getInstantFileNameParser()).thenReturn(INSTANT_FILE_NAME_PARSER); if (!savepoints.isEmpty()) { for (Map.Entry> entry : savepoints.entrySet()) { Pair> savepointMetadataOptionPair = getSavepointMetadata(entry.getValue()); - HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, entry.getKey()); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, entry.getKey()); when(activeTimeline.getInstantDetails(instant)).thenReturn(savepointMetadataOptionPair.getRight()); } } @@ -172,18 +189,20 @@ void testPartitionsForIncrCleaning(boolean isPartitioned, HoodieWriteConfig conf getCleanCommitMetadata(partitionsInLastClean, lastCleanInstant, earliestInstantsInLastClean, lastCompletedTimeInLastClean, savepointsTrackedInLastClean.keySet(), expectedEarliestSavepointInLastClean); HoodieCleanerPlan cleanerPlan = mockLastCleanCommit(mockHoodieTable, lastCleanInstant, earliestInstantsInLastClean, activeTimeline, cleanMetadataOptionPair, savepointsTrackedInLastClean.keySet()); - mockFewActiveInstants(mockHoodieTable, activeInstantsPartitions, savepointsTrackedInLastClean, areCommitsForSavepointsRemoved, replaceCommits); + mockFewActiveInstants(mockHoodieTable, activeTimeline, activeInstantsPartitions, savepointsTrackedInLastClean, areCommitsForSavepointsRemoved, replaceCommits); // mock getAllPartitions HoodieStorage storage = mock(HoodieStorage.class); when(mockHoodieTable.getStorage()).thenReturn(storage); HoodieTableMetadata hoodieTableMetadata = mock(HoodieTableMetadata.class); when(mockHoodieTable.getMetadataTable()).thenReturn(hoodieTableMetadata); + when(mockHoodieTable.getCleanTimeline().filterCompletedInstants().lastInstant()) + .thenReturn(Option.of(INSTANT_GENERATOR.createNewInstant(COMPLETED, HoodieTimeline.CLEAN_ACTION, lastCleanInstant))); when(hoodieTableMetadata.getAllPartitionPaths()).thenReturn(isPartitioned ? Arrays.asList(PARTITION1, PARTITION2, PARTITION3) : Collections.singletonList(StringUtils.EMPTY_STRING)); // Trigger clean and validate partitions to clean and earliest savepoint CleanPlanner cleanPlanner = new CleanPlanner<>(context, mockHoodieTable, config); - HoodieInstant earliestCommitToRetain = new HoodieInstant(COMPLETED, "COMMIT", earliestInstant); + HoodieInstant earliestCommitToRetain = INSTANT_GENERATOR.createNewInstant(COMPLETED, "COMMIT", earliestInstant); List partitionsToClean = cleanPlanner.getPartitionPathsToClean(Option.of(earliestCommitToRetain)); HoodieTableMetaClient metaClient = mock(HoodieTableMetaClient.class); when(metaClient.getActiveTimeline()).thenReturn(activeTimeline); @@ -555,7 +574,7 @@ private static HoodieFileGroup buildFileGroup(List baseFileCommitTimes, String fileGroup = UUID.randomUUID() + "-0"; HoodieFileGroupId fileGroupId = new HoodieFileGroupId(partition, UUID.randomUUID().toString()); HoodieTimeline timeline = mock(HoodieTimeline.class); - when(timeline.lastInstant()).thenReturn(Option.of(new HoodieInstant(COMPLETED, "COMMIT", baseFileCommitTimes.get(baseFileCommitTimes.size() - 1)))); + when(timeline.lastInstant()).thenReturn(Option.of(INSTANT_GENERATOR.createNewInstant(COMPLETED, "COMMIT", baseFileCommitTimes.get(baseFileCommitTimes.size() - 1)))); HoodieFileGroup group = new HoodieFileGroup(fileGroupId, timeline); for (String baseFileCommitTime : baseFileCommitTimes) { when(timeline.containsOrBeforeTimelineStarts(baseFileCommitTime)).thenReturn(true); @@ -609,12 +628,12 @@ private static Pair> getSavepointMetadat private static HoodieCleanerPlan mockLastCleanCommit(HoodieTable hoodieTable, String timestamp, String earliestCommitToRetain, HoodieActiveTimeline activeTimeline, Pair> cleanMetadata, Set savepointsTrackedInLastClean) throws IOException { - HoodieDefaultTimeline cleanTimeline = mock(HoodieDefaultTimeline.class); + BaseTimelineV2 cleanTimeline = mock(BaseTimelineV2.class); when(activeTimeline.getCleanerTimeline()).thenReturn(cleanTimeline); when(hoodieTable.getCleanTimeline()).thenReturn(cleanTimeline); - HoodieDefaultTimeline completedCleanTimeline = mock(HoodieDefaultTimeline.class); + BaseTimelineV2 completedCleanTimeline = mock(BaseTimelineV2.class); when(cleanTimeline.filterCompletedInstants()).thenReturn(completedCleanTimeline); - HoodieInstant latestCleanInstant = new HoodieInstant(COMPLETED, HoodieTimeline.CLEAN_ACTION, timestamp); + HoodieInstant latestCleanInstant = INSTANT_GENERATOR.createNewInstant(COMPLETED, HoodieTimeline.CLEAN_ACTION, timestamp); when(completedCleanTimeline.lastInstant()).thenReturn(Option.of(latestCleanInstant)); when(activeTimeline.isEmpty(latestCleanInstant)).thenReturn(false); when(activeTimeline.getInstantDetails(latestCleanInstant)).thenReturn(cleanMetadata.getRight()); @@ -624,7 +643,7 @@ private static HoodieCleanerPlan mockLastCleanCommit(HoodieTable hoodieTable, St CleanPlanner.LATEST_CLEAN_PLAN_VERSION, null, null, cleanMetadata.getLeft().getExtraMetadata()); when(activeTimeline.readCleanerInfoAsBytes(any())).thenReturn(Option.of(TimelineMetadataUtils.serializeAvroMetadata(cleanerPlan, HoodieCleanerPlan.class).get())); - HoodieDefaultTimeline commitsTimeline = mock(HoodieDefaultTimeline.class); + BaseTimelineV2 commitsTimeline = mock(BaseTimelineV2.class); when(activeTimeline.getCommitsTimeline()).thenReturn(commitsTimeline); when(commitsTimeline.isBeforeTimelineStarts(earliestCommitToRetain)).thenReturn(false); @@ -633,10 +652,10 @@ private static HoodieCleanerPlan mockLastCleanCommit(HoodieTable hoodieTable, St return cleanerPlan; } - private static void mockFewActiveInstants(HoodieTable hoodieTable, Map> activeInstantsToPartitions, + private static void mockFewActiveInstants(HoodieTable hoodieTable, HoodieActiveTimeline activeTimeline, Map> activeInstantsToPartitions, Map> savepointedCommitsToAdd, boolean areCommitsForSavepointsRemoved, List replaceCommits) throws IOException { - HoodieDefaultTimeline commitsTimeline = new HoodieDefaultTimeline(); + BaseTimelineV2 commitsTimeline = new BaseTimelineV2(); List instants = new ArrayList<>(); Map> instantstoProcess = new HashMap<>(); instantstoProcess.putAll(activeInstantsToPartitions); @@ -644,7 +663,7 @@ private static void mockFewActiveInstants(HoodieTable hoodieTable, Map { - HoodieInstant hoodieInstant = new HoodieInstant(COMPLETED, HoodieTimeline.COMMIT_ACTION, k); + HoodieInstant hoodieInstant = INSTANT_GENERATOR.createNewInstant(COMPLETED, HoodieTimeline.COMMIT_ACTION, k); instants.add(hoodieInstant); Map> partitionToWriteStats = new HashMap<>(); v.forEach(partition -> partitionToWriteStats.put(partition, Collections.emptyList())); @@ -653,30 +672,30 @@ private static void mockFewActiveInstants(HoodieTable hoodieTable, Map savepointInstants = savepointedCommitsToAdd.keySet().stream().map(sp -> new HoodieInstant(COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, sp)) + BaseTimelineV2 savepointTimeline = new BaseTimelineV2(); + List savepointInstants = savepointedCommitsToAdd.keySet().stream().map(sp -> INSTANT_GENERATOR.createNewInstant(COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, sp)) .collect(Collectors.toList()); savepointTimeline.setInstants(savepointInstants); - HoodieDefaultTimeline completedReplaceTimeline = new HoodieDefaultTimeline(); - List completedReplaceInstants = replaceCommits.stream().map(rc -> new HoodieInstant(COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, rc)) + BaseTimelineV2 completedReplaceTimeline = new BaseTimelineV2(); + List completedReplaceInstants = replaceCommits.stream().map(rc -> INSTANT_GENERATOR.createNewInstant(COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, rc)) .collect(Collectors.toList()); completedReplaceTimeline.setInstants(completedReplaceInstants); - - when(hoodieTable.getActiveTimeline().findInstantsAfterOrEquals(any())).thenCallRealMethod(); - when(hoodieTable.getActiveTimeline().getCompletedReplaceTimeline()).thenReturn(completedReplaceTimeline); - when(hoodieTable.getActiveTimeline().getSavePointTimeline()).thenReturn(savepointTimeline); + when(activeTimeline.getCompletedReplaceTimeline()).thenReturn(completedReplaceTimeline); + when(activeTimeline.getSavePointTimeline()).thenReturn(savepointTimeline); when(hoodieTable.isPartitioned()).thenReturn(true); when(hoodieTable.isMetadataTable()).thenReturn(false); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/index/TestIndexingCatchupTask.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/index/TestIndexingCatchupTask.java index 95e970f3448c..6fa0dcc08022 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/index/TestIndexingCatchupTask.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/action/index/TestIndexingCatchupTask.java @@ -40,6 +40,7 @@ import java.util.concurrent.CountDownLatch; import java.util.concurrent.atomic.AtomicInteger; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; import static org.mockito.ArgumentMatchers.any; @@ -69,7 +70,7 @@ public void setup() { */ @Test public void testTaskSuccessful() { - List instants = Collections.singletonList(new HoodieInstant(HoodieInstant.State.REQUESTED, "commit", "001")); + List instants = Collections.singletonList(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, "commit", "001")); Set metadataCompletedInstants = new HashSet<>(); AbstractIndexingCatchupTask task = new DummyIndexingCatchupTask( metadataWriter, @@ -90,7 +91,7 @@ public void testTaskSuccessful() { */ @Test public void testTaskInterrupted() { - HoodieInstant neverCompletedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, "commit", "001"); + HoodieInstant neverCompletedInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, "commit", "001"); HoodieActiveTimeline activeTimeline = mock(HoodieActiveTimeline.class); HoodieActiveTimeline filteredTimeline = mock(HoodieActiveTimeline.class); HoodieActiveTimeline furtherFilteredTimeline = mock(HoodieActiveTimeline.class); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/repair/TestRepairUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/repair/TestRepairUtils.java index 12e660477eda..db6d2ce6a5a5 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/repair/TestRepairUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/table/repair/TestRepairUtils.java @@ -47,6 +47,7 @@ import static org.apache.hudi.HoodieTestCommitGenerator.getBaseFilename; import static org.apache.hudi.HoodieTestCommitGenerator.getLogFilename; import static org.apache.hudi.HoodieTestCommitGenerator.initCommitInfoForRepairTests; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -120,15 +121,15 @@ public void testTagInstantsOfBaseAndLogFiles() { public void testGetBaseAndLogFilePathsFromTimeline() throws IOException { setupTimelineInFS(); HoodieTimeline timeline = metaClient.getActiveTimeline(); - HoodieInstant commitInstant = new HoodieInstant( + HoodieInstant commitInstant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); - HoodieInstant inflightInstant = new HoodieInstant( + HoodieInstant inflightInstant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "005"); - HoodieInstant compactionInstant = new HoodieInstant( + HoodieInstant compactionInstant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, "006"); Map>> partitionToFileIdAndNameMap = - instantInfoMap.get(commitInstant.getTimestamp()); + instantInfoMap.get(commitInstant.requestedTime()); Set expectedPaths = partitionToFileIdAndNameMap.entrySet().stream() .flatMap(entry -> entry.getValue().stream() @@ -147,10 +148,10 @@ public void testGetBaseAndLogFilePathsFromTimeline() throws IOException { @Test public void testFindInstantFilesToRemove() throws IOException { setupTimelineInFS(); - HoodieInstant existingInstant = new HoodieInstant( + HoodieInstant existingInstant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); Map>> partitionToFileIdAndNameMap = - instantInfoMap.get(existingInstant.getTimestamp()); + instantInfoMap.get(existingInstant.requestedTime()); List fileListFromFs = partitionToFileIdAndNameMap.entrySet().stream() .flatMap(entry -> entry.getValue().stream() @@ -159,12 +160,12 @@ public void testFindInstantFilesToRemove() throws IOException { .stream() ).collect(Collectors.toList()); String danglingFilePath = new StoragePath("2022/01/02", - getBaseFilename(existingInstant.getTimestamp(), UUID.randomUUID().toString())).toString(); + getBaseFilename(existingInstant.requestedTime(), UUID.randomUUID().toString())).toString(); fileListFromFs.add(danglingFilePath); // Existing instant assertEquals(CollectionUtils.createImmutableList(danglingFilePath), RepairUtils.findInstantFilesToRemove( - existingInstant.getTimestamp(), fileListFromFs, + existingInstant.requestedTime(), fileListFromFs, metaClient.getActiveTimeline(), metaClient.getArchivedTimeline())); // Non-existing instant assertEquals(fileListFromFs, diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java index 1b6c35eb4f75..49c7a4c6e0a5 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/HoodieWriterClientTestHarness.java @@ -23,6 +23,7 @@ import org.apache.hudi.client.BaseHoodieWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; +import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.config.LockConfiguration; import org.apache.hudi.common.data.HoodieData; @@ -44,10 +45,13 @@ import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; @@ -82,6 +86,8 @@ import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.commit.HoodieWriteHelper; import org.apache.hudi.table.marker.WriteMarkersFactory; +import org.apache.hudi.table.upgrade.SupportsUpgradeDowngrade; +import org.apache.hudi.table.upgrade.UpgradeDowngrade; import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.avro.generic.GenericRecord; @@ -116,6 +122,8 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.NULL_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys; import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet; import static org.apache.hudi.config.HoodieClusteringConfig.SCHEDULE_INLINE_CLUSTERING; @@ -269,15 +277,16 @@ protected abstract List writeAndVerifyBatch(BaseHoodieWriteClient protected Object castInsertFirstBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, InstantGenerator instantGenerator) throws Exception { return castInsertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, - isPreppedAPI, assertForCommit, expRecordsInThisCommit, true); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, true, instantGenerator); } protected Object castInsertFirstBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return null; // override in subclasses if needed } @@ -285,9 +294,10 @@ protected Object castWriteBatch(BaseHoodieWriteClient client, String newCommitTi Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, Function3 writeFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + int expTotalCommits, boolean doCommit, InstantGenerator instantGenerator) throws Exception { return castWriteBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, - writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, instantGenerator); } protected Object castWriteBatch(BaseHoodieWriteClient client, String newCommitTime, String prevCommitTime, @@ -295,7 +305,7 @@ protected Object castWriteBatch(BaseHoodieWriteClient client, String newCommitTi Function2, String, Integer> recordGenFunction, Function3 writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return null; // override in subclasses if needed } @@ -303,20 +313,21 @@ protected Object castUpdateBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteC Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return null; // override in subclasses if needed } protected Object castDeleteBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, boolean assertForCommit, - int expRecordsInThisCommit, int expTotalRecords) throws Exception { + int expRecordsInThisCommit, int expTotalRecords, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return castDeleteBatch(writeConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, isPreppedAPI, - assertForCommit, expRecordsInThisCommit, expTotalRecords, true); + assertForCommit, expRecordsInThisCommit, expTotalRecords, true, timelineFactory, instantGenerator); } protected Object castDeleteBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert, + TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return null; // override in subclasses if needed } @@ -377,7 +388,6 @@ protected HoodieWriteConfig getParallelWritingWriteConfig(HoodieFailedWritesClea .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(cleaningPolicy) .withAutoClean(false).build()) - .withTimelineLayoutVersion(1) .withHeartbeatIntervalInMs(3 * 1000) .withFileSystemViewConfig(FileSystemViewStorageConfig.newBuilder() .withRemoteServerPort(timelineServicePort).build()) @@ -417,10 +427,9 @@ protected void verifyRecordsWrittenWithPreservedMetadata(Set commitTimes protected Pair getPartitionAndBaseFilePathsFromLatestCommitMetadata(HoodieTableMetaClient metaClient) throws IOException { String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails( - metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get()).get(), - HoodieCommitMetadata.class); + HoodieInstant instant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get(); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); String filePath = commitMetadata.getPartitionToWriteStats().values().stream() .flatMap(Collection::stream).filter(s -> s.getPath().endsWith(extension)).findAny() .map(HoodieWriteStat::getPath).orElse(null); @@ -555,7 +564,7 @@ protected void verifyClusteredFilesWithReplaceCommitMetadata(String partitionPat List pathInfoList = storage.listDirectEntries(new StoragePath(basePath, partitionPath)); List clusteredFiles = pathInfoList.stream() - .filter(entry -> entry.getPath().getName().contains(replaceCommitInstant.getTimestamp())) + .filter(entry -> entry.getPath().getName().contains(replaceCommitInstant.requestedTime())) .map(pathInfo -> partitionPath + StoragePath.SEPARATOR + pathInfo.getPath().getName()) .collect(Collectors.toList()); assertEquals(clusteredFiles, filesFromReplaceCommit); @@ -573,15 +582,24 @@ protected static List> getAndAssertPen } protected void checkTimelineForUpsertsInternal(HoodieTableMetaClient metaClient) { - HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(metaClient, false); + HoodieActiveTimeline activeTimeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); List instants = activeTimeline.getCommitAndReplaceTimeline().getInstants(); - assertEquals(5, instants.size()); - assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001"), instants.get(0)); - assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "004"), instants.get(1)); + assertEquals(9, instants.size()); + // Should be corresponding to table version 6 + assertEquals(INSTANT_GENERATOR.createNewInstant(REQUESTED, COMMIT_ACTION, "001"), instants.get(0)); + assertEquals(INSTANT_GENERATOR.createNewInstant(INFLIGHT, COMMIT_ACTION, "001"), instants.get(1)); + assertEquals(INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, "001"), instants.get(2)); + assertTrue(instants.get(2).isLegacy()); + assertEquals(INSTANT_GENERATOR.createNewInstant(REQUESTED, COMMIT_ACTION, "004"), instants.get(3)); + assertEquals(INSTANT_GENERATOR.createNewInstant(INFLIGHT, COMMIT_ACTION, "004"), instants.get(4)); + assertEquals(INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, "004"), instants.get(5)); + assertTrue(instants.get(5).isLegacy()); // New Format should have all states of instants - assertEquals(new HoodieInstant(REQUESTED, COMMIT_ACTION, "006"), instants.get(2)); - assertEquals(new HoodieInstant(INFLIGHT, COMMIT_ACTION, "006"), instants.get(3)); - assertEquals(new HoodieInstant(COMPLETED, COMMIT_ACTION, "006"), instants.get(4)); + // Should be corresponding to table version 8 + assertFalse(instants.get(8).isLegacy()); + assertEquals(INSTANT_GENERATOR.createNewInstant(REQUESTED, COMMIT_ACTION, "006"), instants.get(6)); + assertEquals(INSTANT_GENERATOR.createNewInstant(INFLIGHT, COMMIT_ACTION, "006"), instants.get(7)); + assertEquals(INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, "006"), instants.get(8)); } /** @@ -820,7 +838,7 @@ protected void testClustering(HoodieClusteringConfig clusteringConfig, boolean p } if (completeClustering) { String clusteringCommitTime = createMetaClient().reloadActiveTimeline().getCompletedReplaceTimeline() - .getReverseOrderedInstants().findFirst().get().getTimestamp(); + .getReverseOrderedInstants().findFirst().get().requestedTime(); verifyRecordsWritten(clusteringCommitTime, populateMetaFields, allRecords.getLeft().getLeft(), clusterMetadata.getWriteStatuses(), config, createKeyGeneratorFn.apply(config)); } @@ -857,17 +875,17 @@ protected void testCommitWritesRelativePaths(Function transformInputFn) throws E // Get base file paths from commit metadata String actionType = metaClient.getCommitActionType(); - HoodieInstant commitInstant = new HoodieInstant(false, actionType, instantTime); + HoodieInstant commitInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, actionType, instantTime); HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(commitInstant, commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class); StoragePath basePath = metaClient.getBasePath(); Collection commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values(); // Read from commit file - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( - metaClient.reloadActiveTimeline() - .getInstantDetails(new HoodieInstant(COMPLETED, COMMIT_ACTION, instantTime)).get(), + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, + metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); HashMap paths = metadata.getFileIdAndFullPaths(basePath); // Compare values in both to make sure they are equal. @@ -890,8 +908,9 @@ protected void testMetadataStatsOnCommit(boolean populateMetaFields, Function tr assertTrue(testTable.commitExists(instantTime0), "After explicit commit, commit file should be created"); // Read from commit file - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( - createMetaClient().reloadActiveTimeline().getInstantDetails(new HoodieInstant(COMPLETED, COMMIT_ACTION, instantTime0)).get(), + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime0); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, + createMetaClient().reloadActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -909,9 +928,10 @@ protected void testMetadataStatsOnCommit(boolean populateMetaFields, Function tr assertTrue(testTable.commitExists(instantTime1), "After explicit commit, commit file should be created"); metaClient = createMetaClient(); + instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime1); // Read from commit file - metadata = HoodieCommitMetadata.fromBytes( - metaClient.reloadActiveTimeline().getInstantDetails(new HoodieInstant(COMPLETED, COMMIT_ACTION, instantTime1)).get(), + metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, + metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); inserts = 0; int upserts = 0; @@ -1032,14 +1052,14 @@ protected void testDeduplication(Function3, BaseHoodieWriteCli * @throws Exception in case of failure */ protected void testAutoCommit(Function3 writeFn, - boolean isPrepped, boolean populateMetaFields) throws Exception { + boolean isPrepped, boolean populateMetaFields, InstantGenerator instantGenerator) throws Exception { HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder().withAutoCommit(false); // Set autoCommit false addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); try (BaseHoodieWriteClient client = getHoodieWriteClient(cfgBuilder.build())) { String prevCommitTime = "000"; String newCommitTime = "001"; int numRecords = 200; - Object result = castInsertFirstBatch(cfgBuilder.build(), client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords); + Object result = castInsertFirstBatch(cfgBuilder.build(), client, newCommitTime, prevCommitTime, numRecords, writeFn, isPrepped, false, numRecords, instantGenerator); assertFalse(testTable.commitExists(newCommitTime), "If Autocommit is false, then commit should not be made automatically"); assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); assertTrue(testTable.commitExists(newCommitTime), "After explicit commit, commit file should be created"); @@ -1052,7 +1072,7 @@ protected void testAutoCommit(Function3 writeFn, boolean populateMetaFields, boolean isPrepped) throws Exception { + protected void testUpsertsInternal(Function3 writeFn, boolean populateMetaFields, boolean isPrepped, + SupportsUpgradeDowngrade upgradeDowngrade) throws Exception { metaClient = createMetaClient(); - HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withRollbackUsingMarkers(true); + HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(HoodieFailedWritesCleaningPolicy.LAZY).withRollbackUsingMarkers(true) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()); addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields); // Force using older timeline layout - HoodieWriteConfig config = cfgBuilder.withTimelineLayoutVersion(VERSION_0).build(); HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) - .setTimelineLayoutVersion(VERSION_0) + .setTableVersion(6) .setPopulateMetaFields(populateMetaFields) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); + metaClient = HoodieTestUtils.createMetaClient(storageConf, new StoragePath(basePath), HoodieTableVersion.SIX); + HoodieWriteConfig config = cfgBuilder.withWriteTableVersion(6).build(); BaseHoodieWriteClient client = getHoodieWriteClient(config); // Write 1 (only inserts) @@ -1169,7 +1192,7 @@ protected void testUpsertsInternal(Function3, String, Integ castWriteBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, // unused as genFn uses hard-coded number of inserts/updates/deletes - -1, recordGenFunction, BaseHoodieWriteClient::upsert, true, 200, 200, 1, false, true); + -1, recordGenFunction, BaseHoodieWriteClient::upsert, true, 200, 200, 1, false, true, INSTANT_GENERATOR); /** * Write 2 (deletes+writes). @@ -1253,7 +1278,7 @@ protected void testDeletes(Function3, String, Integ recordGenFunction = secondBatchGenFn.apply(newCommitTime, numRecordsInSecondBatch, recordsInFirstBatch); castWriteBatch(client, newCommitTime, prevCommitTime, Option.empty(), initCommitTime, numRecordsInSecondBatch, recordGenFunction, - BaseHoodieWriteClient::upsert, true, expRecordsInSecondBatch, expTotalRecords, 2, false, true); + BaseHoodieWriteClient::upsert, true, expRecordsInSecondBatch, expTotalRecords, 2, false, true, INSTANT_GENERATOR); } protected void testRollbackFailedCommits(boolean populateMetaFields) throws Exception { @@ -1265,17 +1290,17 @@ protected void testRollbackFailedCommits(boolean populateMetaFields) throws Exce // perform 1 successful commit castWriteBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, true); + 0, true, INSTANT_GENERATOR); // Perform 2 failed writes to table castWriteBatch(client, "200", "100", Option.of(Arrays.asList("200")), "100", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); // refresh data generator to delete records generated from failed commits dataGen = new HoodieTestDataGenerator(); @@ -1283,7 +1308,7 @@ protected void testRollbackFailedCommits(boolean populateMetaFields) throws Exce client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, true); + 0, true, INSTANT_GENERATOR); HoodieTableMetaClient metaClient = createMetaClient(); assertEquals(0, metaClient.getActiveTimeline().getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION)).countInstants()); @@ -1300,7 +1325,7 @@ protected void testRollbackFailedCommits(boolean populateMetaFields) throws Exce // Perform 1 successful write castWriteBatch(client, "500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, true); + 0, true, INSTANT_GENERATOR); client.clean(); client.close(); HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload(); @@ -1326,12 +1351,12 @@ protected void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMet // Perform 1 successful writes to table castWriteBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, true); + 0, true, INSTANT_GENERATOR); // Perform 1 failed writes to table castWriteBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); // Toggle cleaning policy to LAZY cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY; @@ -1339,12 +1364,12 @@ protected void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMet client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); // Wait till enough time passes such that the 2 failed commits heartbeats are expired boolean conditionMet = false; @@ -1359,12 +1384,12 @@ protected void testRollbackFailedCommitsToggleCleaningPolicy(boolean populateMet client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "500", "400", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "600", "500", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 300, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); // Toggle cleaning policy to EAGER cleaningPolicy = EAGER; @@ -1385,24 +1410,24 @@ protected void testParallelInsertAndCleanPreviousFailedCommits(boolean populateM // perform 1 successful write castWriteBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 100, - 0, true); + 0, true, INSTANT_GENERATOR); // Perform 2 failed writes to table castWriteBatch(client, "200", "100", Option.of(Arrays.asList("200")), "200", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 100, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); client = getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)); castWriteBatch(client, "300", "200", Option.of(Arrays.asList("300")), "300", 100, dataGen::generateInserts, BaseHoodieWriteClient::bulkInsert, false, 100, 100, - 0, false); + 0, false, INSTANT_GENERATOR); client.close(); // refresh data generator to delete records generated from failed commits dataGen = new HoodieTestDataGenerator(); // Create a successful commit Future commit3 = service.submit(() -> castWriteBatch(getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), "400", "300", Option.of(Arrays.asList("400")), "300", 100, dataGen::generateInserts, - BaseHoodieWriteClient::bulkInsert, false, 100, 100, 0, true)); + BaseHoodieWriteClient::bulkInsert, false, 100, 100, 0, true, INSTANT_GENERATOR)); commit3.get(); HoodieTableMetaClient metaClient = createMetaClient(); @@ -1418,7 +1443,7 @@ protected void testParallelInsertAndCleanPreviousFailedCommits(boolean populateM } Future commit4 = service.submit(() -> castWriteBatch(getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)), "500", "400", Option.of(Arrays.asList("500")), "500", 100, dataGen::generateInserts, - BaseHoodieWriteClient::bulkInsert, false, 100, 100, 0, true)); + BaseHoodieWriteClient::bulkInsert, false, 100, 100, 0, true, INSTANT_GENERATOR)); Future clean1 = service.submit(() -> getHoodieWriteClient(getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)).clean()); commit4.get(); clean1.get(); diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java index f050f91d1692..7b27a1ea0168 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/utils/TestMetadataConversionUtils.java @@ -66,6 +66,8 @@ import java.util.Random; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -83,7 +85,7 @@ public void testCompletedClean() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createCleanMetadata(newCommitTime); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, newCommitTime), metaClient); assertEquals(State.COMPLETED.toString(), metaEntry.getActionState()); assertEquals(newCommitTime, metaEntry.getHoodieCleanMetadata().getStartCleanTime()); } @@ -93,7 +95,7 @@ public void testCompletedReplace() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE, true); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); assertEquals(metaEntry.getHoodieReplaceCommitMetadata().getOperationType(), WriteOperationType.INSERT_OVERWRITE.toString()); } @@ -103,7 +105,7 @@ public void testEmptyRequestedReplace() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE_TABLE, false); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.REQUESTED.toString()); assertNull(metaEntry.getHoodieRequestedReplaceMetadata()); } @@ -113,7 +115,7 @@ public void testEmptyInflightReplace() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE_TABLE, true); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.INFLIGHT.toString()); assertNull(metaEntry.getHoodieInflightReplaceMetadata()); } @@ -123,7 +125,7 @@ public void testNonEmptyInflightReplace() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE_TABLE, false); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.INFLIGHT.toString()); assertEquals(metaEntry.getHoodieInflightReplaceMetadata().getOperationType(), WriteOperationType.INSERT_OVERWRITE_TABLE.name()); } @@ -133,7 +135,7 @@ public void testCompletedCommit() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createCommitMetadata(newCommitTime); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.INSERT.toString()); } @@ -143,7 +145,7 @@ public void testCompletedDeltaCommit() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createDeltaCommitMetadata(newCommitTime); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); assertEquals(metaEntry.getActionType(), HoodieTimeline.DELTA_COMMIT_ACTION); } @@ -153,7 +155,7 @@ public void testCompletedRollback() throws Exception { String newCommitTime = HoodieTestTable.makeNewCommitTime(); createRollbackMetadata(newCommitTime); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); assertEquals(metaEntry.getHoodieRollbackMetadata().getStartRollbackTime(), newCommitTime); } @@ -167,7 +169,7 @@ public void testCompletedCompaction() throws Exception { // So we have to reload here to get all commits written before. metaClient.reloadActiveTimeline(); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), metaClient); assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); assertEquals(metaEntry.getHoodieCommitMetadata().getOperationType(), WriteOperationType.COMPACT.toString()); } @@ -217,8 +219,9 @@ public void testArchivedInsertOverwriteWithoutClustering() throws Exception { assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived.getAction()); assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieReplaceCommitMetadata.class)); - assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived.getPlan().array(), HoodieCommitMetadata.class), - "Insert overwrite without clustering should have a plan"); + assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), + archived.getPlan().array(), HoodieCommitMetadata.class), "Insert overwrite without clustering should have a plan"); String newCommitTime2 = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime2, WriteOperationType.INSERT_OVERWRITE_TABLE, false); @@ -227,7 +230,9 @@ public void testArchivedInsertOverwriteWithoutClustering() throws Exception { assertEquals(newCommitTime2, archived2.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived2.getAction()); assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived2.getMetadata().array(), HoodieReplaceCommitMetadata.class)); - assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived2.getPlan().array(), HoodieCommitMetadata.class), + assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime2), + archived2.getPlan().array(), HoodieCommitMetadata.class), "Insert overwrite table without clustering should have a plan"); } @@ -259,7 +264,9 @@ public void testArchivedCommit() throws Exception { HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); - assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieCommitMetadata.class)); + assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), + archived.getMetadata().array(), HoodieCommitMetadata.class)); } @Test @@ -290,7 +297,9 @@ public void testArchivedCompaction() throws Exception { HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); - assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieCommitMetadata.class)); + assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), + archived.getMetadata().array(), HoodieCommitMetadata.class)); assertDoesNotThrow(() -> CompactionUtils.getCompactionPlan(metaClient, Option.of(archived.getPlan().array()))); } @@ -406,8 +415,8 @@ HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEF } private ActiveAction getActiveInstant(String instantTime) { - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - List instants = rawActiveTimeline.getInstantsAsStream().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList()); + HoodieActiveTimeline rawActiveTimeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); + List instants = rawActiveTimeline.getInstantsAsStream().filter(instant -> instant.requestedTime().equals(instantTime)).collect(Collectors.toList()); return ActiveAction.fromInstants(instants); } } diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java index 019f6d3a9a3c..5fac2d9a0374 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/HoodieFlinkTableServiceClient.java @@ -31,7 +31,7 @@ import org.apache.hudi.common.model.TableServiceType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -81,7 +81,7 @@ protected HoodieWriteMetadata> compact(String compactionInstan protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable table, String compactionCommitTime) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction: " + config.getTableName()); List writeStats = metadata.getWriteStats(); - final HoodieInstant compactionInstant = HoodieTimeline.getCompactionInflightInstant(compactionCommitTime); + final HoodieInstant compactionInstant = table.getInstantGenerator().getCompactionInflightInstant(compactionCommitTime); try { this.txnManager.beginTransaction(Option.of(compactionInstant), Option.empty()); finalizeWrite(table, compactionCommitTime, writeStats); @@ -100,7 +100,7 @@ protected void completeCompaction(HoodieCommitMetadata metadata, HoodieTable tab if (compactionTimer != null) { long durationInMs = metrics.getDurationInMs(compactionTimer.stop()); try { - metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(compactionCommitTime).getTime(), + metrics.updateCommitMetrics(TimelineUtils.parseDateFromInstantTime(compactionCommitTime).getTime(), durationInMs, metadata, HoodieActiveTimeline.COMPACTION_ACTION); } catch (ParseException e) { throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " @@ -116,7 +116,7 @@ protected void completeClustering( String clusteringCommitTime, Option> writeStatuses) { this.context.setJobStatus(this.getClass().getSimpleName(), "Collect clustering write status and commit clustering"); - HoodieInstant clusteringInstant = ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline()).get(); + HoodieInstant clusteringInstant = ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline(), table.getInstantGenerator()).get(); List writeStats = metadata.getPartitionToWriteStats().entrySet().stream().flatMap(e -> e.getValue().stream()).collect(Collectors.toList()); if (writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum() > 0) { @@ -141,7 +141,7 @@ protected void completeClustering( ClusteringUtils.transitionClusteringOrReplaceInflightToComplete( false, clusteringInstant, - serializeCommitMetadata(metadata), + serializeCommitMetadata(table.getMetaClient().getCommitMetadataSerDe(), metadata), table.getActiveTimeline()); } catch (IOException e) { throw new HoodieClusteringException( @@ -155,7 +155,7 @@ protected void completeClustering( if (clusteringTimer != null) { long durationInMs = metrics.getDurationInMs(clusteringTimer.stop()); try { - metrics.updateCommitMetrics(HoodieActiveTimeline.parseDateFromInstantTime(clusteringCommitTime).getTime(), + metrics.updateCommitMetrics(TimelineUtils.parseDateFromInstantTime(clusteringCommitTime).getTime(), durationInMs, metadata, HoodieActiveTimeline.CLUSTERING_ACTION); } catch (ParseException e) { throw new HoodieCommitException("Commit time is not of valid format. Failed to commit compaction " @@ -203,7 +203,7 @@ public void initMetadataTable() { HoodieFlinkTable table = (HoodieFlinkTable) createTable(config, storageConf, false); if (config.isMetadataTableEnabled()) { Option latestPendingInstant = table.getActiveTimeline() - .filterInflightsAndRequested().lastInstant().map(HoodieInstant::getTimestamp); + .filterInflightsAndRequested().lastInstant().map(HoodieInstant::requestedTime); try { // initialize the metadata table path // guard the metadata writer with concurrent lock diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategyRecently.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategyRecently.java index 7d2c5785527c..10046f84f5bf 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategyRecently.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/client/clustering/plan/strategy/FlinkSizeBasedClusteringPlanStrategyRecently.java @@ -118,8 +118,8 @@ private List getPartitionPathInActiveTimeline(HoodieTable { try { - HoodieCommitMetadata metadata = - HoodieCommitMetadata.fromBytes(cowCommitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = hoodieTable.getMetaClient().getCommitMetadataSerDe().deserialize(instant, + cowCommitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); partitions.addAll(metadata.getWritePartitionPaths()); } catch (IOException e) { // ignore Exception here diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java index 36d19b98de53..aa5dfc98f113 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/metadata/FlinkHoodieBackedTableMetadataWriter.java @@ -142,7 +142,7 @@ protected void commitInternal(String instantTime, Map alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)) + Option alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.requestedTime().equals(instantTime)) .lastInstant(); LOG.info(String.format("%s completed commit at %s being applied to MDT.", alreadyCompletedInstant.isPresent() ? "Already" : "Partially", instantTime)); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java index 765da293edf1..c902b73420ae 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/HoodieFlinkTable.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -57,7 +56,6 @@ public static HoodieFlinkTable create(HoodieWriteConfig config, HoodieEng HoodieTableMetaClient.builder() .setConf(context.getStorageConf().newInstance()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setTimeGeneratorConfig(config.getTimeGeneratorConfig()) .setFileSystemRetryConfig(config.getFileSystemRetryConfig()).build(); return HoodieFlinkTable.create(config, context, metaClient); diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java index 90d12a2c4963..0d80e26fff9d 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/commit/FlinkDeletePartitionCommitActionExecutor.java @@ -79,9 +79,9 @@ public HoodieWriteMetadata> execute() { // created requested HoodieInstant dropPartitionsInstant = - new HoodieInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime); + instantGenerator.createNewInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime); if (!table.getStorage().exists(new StoragePath( - table.getMetaClient().getMetaPath(), dropPartitionsInstant.getFileName()))) { + table.getMetaClient().getMetaPath(), instantFileNameGenerator.getFileName(dropPartitionsInstant)))) { HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setOperationType(WriteOperationType.DELETE_PARTITION.name()) diff --git a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java index 058459b1bf62..0f1f9fe5e724 100644 --- a/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-flink-client/src/main/java/org/apache/hudi/table/action/compact/HoodieFlinkMergeOnReadTableCompactor.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTable; @@ -45,9 +46,10 @@ public class HoodieFlinkMergeOnReadTableCompactor @Override public void preCompact( HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) { + InstantGenerator instantGenerator = table.getInstantGenerator(); HoodieInstant inflightInstant = WriteOperationType.COMPACT.equals(operationType) - ? HoodieTimeline.getCompactionInflightInstant(instantTime) - : HoodieTimeline.getLogCompactionInflightInstant(instantTime); + ? instantGenerator.getCompactionInflightInstant(instantTime) + : instantGenerator.getLogCompactionInflightInstant(instantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java index 0493bb41593e..8dfbcae32ee1 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaTableServiceClient.java @@ -49,7 +49,7 @@ protected HoodieJavaTableServiceClient(HoodieEngineContext context, protected void validateClusteringCommit(HoodieWriteMetadata> clusteringMetadata, String clusteringCommitTime, HoodieTable table) { if (clusteringMetadata.getWriteStatuses().isEmpty()) { HoodieClusteringPlan clusteringPlan = ClusteringUtils.getClusteringPlan( - table.getMetaClient(), ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline()).get()) + table.getMetaClient(), ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline(), table.getInstantGenerator()).get()) .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException( "Unable to read clustering plan for instant: " + clusteringCommitTime)); throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + clusteringCommitTime diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java index bba0e9f840af..968454a1b3db 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/client/HoodieJavaWriteClient.java @@ -172,7 +172,7 @@ public List bulkInsert(List> records, public void transitionInflight(String instantTime) { HoodieTableMetaClient metaClient = createMetaClient(true); metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(HoodieInstant.State.REQUESTED, metaClient.getCommitActionType(), instantTime), + metaClient.createNewInstant(HoodieInstant.State.REQUESTED, metaClient.getCommitActionType(), instantTime), Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java index 76e46449abf2..a0ed4e65747c 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/HoodieJavaTable.java @@ -25,7 +25,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -52,7 +51,6 @@ public static HoodieJavaTable create(HoodieWriteConfig config, HoodieEngi HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setConf(context.getStorageConf().newInstance()).setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setTimeGeneratorConfig(config.getTimeGeneratorConfig()).build(); return HoodieJavaTable.create(config, context, metaClient); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java index 9d61eb8359d9..ab1683a98534 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/cluster/JavaExecuteClusteringCommitActionExecutor.java @@ -46,7 +46,7 @@ public JavaExecuteClusteringCommitActionExecutor(HoodieEngineContext context, String instantTime) { super(context, config, table, instantTime, WriteOperationType.CLUSTER); this.clusteringPlan = ClusteringUtils.getClusteringPlan( - table.getMetaClient(), ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline()).get()) + table.getMetaClient(), ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline(), table.getInstantGenerator()).get()) .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException( "Unable to read clustering plan for instant: " + instantTime)); } diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java index 61cffe26d68f..054f0342ccfe 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/BaseJavaCommitActionExecutor.java @@ -97,10 +97,10 @@ public HoodieWriteMetadata> execute(List> inpu saveWorkloadProfileMetadataToInflight(workloadProfile, instantTime); } catch (Exception e) { HoodieTableMetaClient metaClient = table.getMetaClient(); - HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, metaClient.getCommitActionType(), instantTime); try { if (!metaClient.getStorage().exists( - new StoragePath(metaClient.getMetaPath(), inflightInstant.getFileName()))) { + new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(inflightInstant)))) { throw new HoodieCommitException("Failed to commit " + instantTime + " unable to save inflight metadata ", e); } } catch (IOException ex) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java index 5503573656c6..d9905ef7fdca 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaBulkInsertHelper.java @@ -73,7 +73,7 @@ public HoodieWriteMetadata> bulkInsert(final List getSmallFiles(String partitionPath) { if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); List allFiles = table.getBaseFileOnlyView() - .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.requestedTime()).collect(Collectors.toList()); for (HoodieBaseFile file : allFiles) { if (file.getFileSize() < config.getParquetSmallFileLimit()) { @@ -320,8 +321,9 @@ protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, Hoodi Iterator instants = commitTimeline.getReverseOrderedInstants().iterator(); while (instants.hasNext()) { HoodieInstant instant = instants.next(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() + .deserialize(instant, commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); long totalBytesWritten = commitMetadata.fetchTotalBytesWritten(); long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten(); if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) { diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java index d8994ce02c39..8e9c479928c2 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/compact/HoodieJavaMergeOnReadTableCompactor.java @@ -46,7 +46,7 @@ public void preCompact( if (WriteOperationType.LOG_COMPACT.equals(operationType)) { throw new UnsupportedOperationException("Log compaction is not supported for this execution engine."); } - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(instantTime); + HoodieInstant inflightInstant = table.getInstantGenerator().getCompactionInflightInstant(instantTime); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { table.rollbackInflightCompaction(inflightInstant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java index 8a2f1f07e7ab..78eeaf72d3d7 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/TestJavaHoodieBackedMetadata.java @@ -151,10 +151,13 @@ import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getNextCommitTime; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataTable; @@ -356,26 +359,26 @@ public void testMetadataTableArchival() throws Exception { // and the "00000000000000" init deltacommit should be archived. HoodieTableMetaClient metadataMetaClient = createMetaClientForMetadataTable(); HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); // Trigger clustering in the data table, archival should not kick in, even though conditions are met. doCluster(testTable, metaClient.createNewInstantTime()); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); getHoodieWriteClient(writeConfig); // Trigger a regular write operation. data set timeline archival should kick in. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); archiveDataTable(writeConfig, createMetaClient()); assertEquals(instants.get(3), - metaClient.reloadActiveTimeline().getCommitsTimeline().firstInstant().get().getTimestamp()); + metaClient.reloadActiveTimeline().getCommitsTimeline().firstInstant().get().requestedTime()); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); // Trigger a regular write operation. metadata timeline archival should kick in. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(3), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(3), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); } @ParameterizedTest @@ -410,7 +413,7 @@ public void testMetadataArchivalCleanConfig(HoodieTableType tableType) throws Ex // and the "00000000000000" init deltacommit should be archived. HoodieTableMetaClient metadataMetaClient = createMetaClientForMetadataTable(); HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); getHoodieWriteClient(writeConfig); // Trigger data table archive, should archive 1st, 2nd. @@ -418,7 +421,7 @@ public void testMetadataArchivalCleanConfig(HoodieTableType tableType) throws Ex // Trigger a regular write operation. metadata timeline archival should kick in and catch up with data table. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(2), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(2), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); } @ParameterizedTest @@ -748,7 +751,7 @@ public void testMetadataRollbackWithCompaction() throws Exception { // collect all commit meta files from metadata table. HoodieTableMetaClient metadataMetaClient = HoodieTestUtils.init(storageConf, HoodieTableMetadata.getMetadataTableBasePath(basePath), tableType, new Properties()); - String completionTimeForCommit3 = metadataMetaClient.getActiveTimeline().filter(instant -> instant.getTimestamp().equals(newCommitTime3)).firstInstant() + String completionTimeForCommit3 = metadataMetaClient.getActiveTimeline().filter(instant -> instant.requestedTime().equals(newCommitTime3)).firstInstant() .map(HoodieInstant::getCompletionTime) .orElseThrow(() -> new IllegalStateException(newCommitTime3 + " should exist on the metadata")); String completionTimeForRollback = metadataMetaClient.getActiveTimeline().filter(instant -> instant.getAction().equals(ROLLBACK_ACTION)).firstInstant() @@ -757,7 +760,7 @@ public void testMetadataRollbackWithCompaction() throws Exception { // ensure commit2's delta commit in MDT has completion time > the actual rollback for previous failed commit i.e. commit2. // if rollback wasn't eager, rollback's completion time will be lower than the commit3'd delta commit completion time. - assertTrue(HoodieTimeline.compareTimestamps(completionTimeForCommit3, GREATER_THAN, completionTimeForRollback)); + assertTrue(compareTimestamps(completionTimeForCommit3, GREATER_THAN, completionTimeForRollback)); } } @@ -1110,7 +1113,7 @@ public void testManualRollbacks() throws Exception { List allInstants = metaClient.reloadActiveTimeline().getCommitsTimeline().getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant instantToRollback : allInstants) { try { - testTable.doRollback(instantToRollback.getTimestamp(), metaClient.createNewInstantTime()); + testTable.doRollback(instantToRollback.requestedTime(), metaClient.createNewInstantTime()); validateMetadata(testTable); } catch (HoodieMetadataException e) { // This is expected since we are rolling back commits that are older than the latest compaction on the MDT @@ -1433,9 +1436,9 @@ public void testColStatsPrefixLookup() throws IOException { // populate commit -> partition -> file info to assist in validation and prefix search metaClient.getActiveTimeline().getInstants().forEach(entry -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); - String commitTime = entry.getTimestamp(); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(entry, metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); + String commitTime = entry.requestedTime(); if (!commitToPartitionsToFiles.containsKey(commitTime)) { commitToPartitionsToFiles.put(commitTime, new HashMap<>()); } @@ -1562,7 +1565,7 @@ public void testEagerRollbackinMDT() throws IOException { .collect(Collectors.toList()); List rollbackFiles = metaFiles.stream() .filter(fileStatus -> - fileStatus.getPath().getName().equals(rollbackInstant.getFileName())) + fileStatus.getPath().getName().equals(INSTANT_FILE_NAME_GENERATOR.getFileName(rollbackInstant))) .collect(Collectors.toList()); // ensure commit3's delta commit in MDT has last mod time > the actual rollback for previous failed commit i.e. commit2. @@ -1724,9 +1727,9 @@ public void testMetadataMultiWriter() throws Exception { // Ensure all commits were synced to the Metadata Table HoodieTableMetaClient metadataMetaClient = createMetaClientForMetadataTable(); assertEquals(metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 5); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000002"))); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000003"))); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000004"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000002"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000003"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000004"))); // Compaction may occur if the commits completed in order assertTrue(metadataMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants() <= 1); @@ -2112,7 +2115,8 @@ public void testCleaningArchivingAndCompaction() throws Exception { public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { init(HoodieTableType.COPY_ON_WRITE, false); HoodieEngineContext engineContext = new HoodieJavaEngineContext(storageConf); - + // set hoodie.table.version to 6 in hoodie.properties file + changeTableVersion(HoodieTableVersion.SIX); // Perform a commit. This should bootstrap the metadata table with latest version. List records; List writeStatuses; @@ -2120,6 +2124,7 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); + properties.setProperty("hoodie.write.table.version", "6"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(false, true, false) .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) @@ -2152,6 +2157,7 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { // set hoodie.table.version to 2 in hoodie.properties file changeTableVersion(HoodieTableVersion.TWO); + properties.setProperty("hoodie.write.table.version", "8"); writeConfig = getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) @@ -2214,7 +2220,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = - metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile( new StoragePath(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName))); } @@ -2317,7 +2323,7 @@ public void testErrorCases() throws Exception { // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = - metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get().getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile( new StoragePath(basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName))); } @@ -2463,8 +2469,8 @@ public void testRepeatedActionWithSameInstantTime() throws Exception { // To simulate failed clean on the main dataset, we will delete the completed clean instant String cleanInstantFileName = - metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() - .getReverseOrderedInstants().findFirst().get().getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() + .getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile(new StoragePath( basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, cleanInstantFileName))); assertEquals( diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorage.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorage.java index 3baf26257b8e..9fda8f722eea 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorage.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnCopyOnWriteStorage.java @@ -31,6 +31,8 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -43,6 +45,7 @@ import org.apache.hudi.keygen.BaseKeyGenerator; import org.apache.hudi.keygen.factory.HoodieAvroKeyGeneratorFactory; import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.upgrade.JavaUpgradeDowngradeHelper; import org.apache.hudi.testutils.HoodieJavaClientTestHarness; import org.junit.jupiter.api.BeforeEach; @@ -60,6 +63,7 @@ import java.util.function.Function; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.fail; @@ -90,10 +94,11 @@ public void setUpTestTable() { protected Object castInsertFirstBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return insertFirstBatch(writeConfig, (HoodieJavaWriteClient) client, newCommitTime, initCommitTime, numRecordsInThisCommit, (writeClient, records, commitTime) -> (List) writeFn.apply(writeClient, records, commitTime), - isPreppedAPI, assertForCommit, expRecordsInThisCommit, filterForCommitTimeWithAssert); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, filterForCommitTimeWithAssert, instantGenerator); } @Override @@ -102,10 +107,10 @@ protected Object castWriteBatch(BaseHoodieWriteClient client, String newCommitTi Function2, String, Integer> recordGenFunction, Function3 writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return writeBatch((HoodieJavaWriteClient) client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, (writeClient, records, commitTime) -> (List) writeFn.apply(writeClient, records, commitTime), - assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, filterForCommitTimeWithAssert); + assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } @Override @@ -113,21 +118,22 @@ protected Object castUpdateBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteC Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); return writeBatch((HoodieJavaWriteClient) client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, (writeClient, records, commitTime) -> (List) writeFn.apply(writeClient, records, commitTime), assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, false, filterForCommitTimeWithAssert); + expTotalCommits, false, filterForCommitTimeWithAssert, instantGenerator); } @Override protected Object castDeleteBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + boolean filterForCommitTimeWithAssert, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return deleteBatch(writeConfig, (HoodieJavaWriteClient) client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, - isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, timelineFactory, instantGenerator); } @Override @@ -145,7 +151,7 @@ protected String[] assertTheEntireDatasetHasAllRecordsStill(int expectedRecords) */ @Test public void testAutoCommitOnInsert() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insert(recordRDD, instantTime), false, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insert(recordRDD, instantTime), false, true, INSTANT_GENERATOR); } /** @@ -153,7 +159,7 @@ public void testAutoCommitOnInsert() throws Exception { */ @Test public void testAutoCommitOnInsertPrepped() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insertPreppedRecords(recordRDD, instantTime), true, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insertPreppedRecords(recordRDD, instantTime), true, true, INSTANT_GENERATOR); } /** @@ -161,7 +167,7 @@ public void testAutoCommitOnInsertPrepped() throws Exception { */ @Test public void testAutoCommitOnUpsert() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), false, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), false, true, INSTANT_GENERATOR); } /** @@ -169,7 +175,7 @@ public void testAutoCommitOnUpsert() throws Exception { */ @Test public void testAutoCommitOnUpsertPrepped() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true, INSTANT_GENERATOR); } /** @@ -177,7 +183,7 @@ public void testAutoCommitOnUpsertPrepped() throws Exception { */ @Test public void testAutoCommitOnBulkInsert() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime), false, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime), false, true, INSTANT_GENERATOR); } /** @@ -186,7 +192,7 @@ public void testAutoCommitOnBulkInsert() throws Exception { @Test public void testAutoCommitOnBulkInsertPrepped() throws Exception { testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, - Option.empty()), true, true); + Option.empty()), true, true, INSTANT_GENERATOR); } /** @@ -226,7 +232,7 @@ public void testDeduplicationOnUpsert() throws Exception { */ @Test public void testUpserts() throws Exception { - testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), true, false); + testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), true, false, JavaUpgradeDowngradeHelper.getInstance()); } /** @@ -234,7 +240,7 @@ public void testUpserts() throws Exception { */ @Test public void testUpsertsPrepped() throws Exception { - testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true); + testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true, JavaUpgradeDowngradeHelper.getInstance()); } @Override @@ -284,7 +290,7 @@ protected void testMergeHandle(HoodieWriteConfig config) throws IOException { */ @Test public void testInsertsWithHoodieConcatHandle() throws Exception { - testHoodieConcatHandle(true, false); + testHoodieConcatHandle(true, false, INSTANT_GENERATOR); } /** @@ -292,7 +298,7 @@ public void testInsertsWithHoodieConcatHandle() throws Exception { */ @Test public void testInsertsPreppedWithHoodieConcatHandle() throws Exception { - testHoodieConcatHandle(true, true); + testHoodieConcatHandle(true, true, INSTANT_GENERATOR); } /** @@ -300,7 +306,7 @@ public void testInsertsPreppedWithHoodieConcatHandle() throws Exception { */ @Test public void testInsertsWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Exception { - testHoodieConcatHandleOnDupInserts(false); + testHoodieConcatHandleOnDupInserts(false, INSTANT_GENERATOR); } /** @@ -308,7 +314,7 @@ public void testInsertsWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Ex */ @Test public void testInsertsPreppedWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Exception { - testHoodieConcatHandleOnDupInserts(true); + testHoodieConcatHandleOnDupInserts(true, INSTANT_GENERATOR); } /** @@ -337,7 +343,7 @@ public void testDeletes() throws Exception { */ @Test public void testDeletesForInsertsInSameBatch() throws Exception { - super.testDeletesForInsertsInSameBatch(); + super.testDeletesForInsertsInSameBatch(INSTANT_GENERATOR); } @Test diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnMergeOnReadStorage.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnMergeOnReadStorage.java index cf3d81a3e134..1e34a2336133 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnMergeOnReadStorage.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/client/functional/TestHoodieJavaClientOnMergeOnReadStorage.java @@ -36,6 +36,8 @@ import java.util.Arrays; import java.util.Map; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertDataInMORTable; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -61,20 +63,20 @@ public void testReadingMORTableWithoutBaseFile() throws Exception { // Insert String commitTime = client.createNewInstantTime(); insertBatch(config, client, commitTime, "000", 100, HoodieJavaWriteClient::insert, - false, false, 100, 100, 1, Option.empty()); + false, false, 100, 100, 1, Option.empty(), INSTANT_GENERATOR); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, HoodieJavaWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Delete 5 records String prevCommitTime = commitTime; commitTime = client.createNewInstantTime(); deleteBatch(config, client, commitTime, prevCommitTime, "000", 25, false, false, - 0, 100); + 0, 100, TIMELINE_FACTORY, INSTANT_GENERATOR); // Verify all the records. metaClient.reloadActiveTimeline(); @@ -94,14 +96,14 @@ public void testCompactionOnMORTable() throws Exception { // Insert String commitTime = client.createNewInstantTime(); insertBatch(config, client, commitTime, "000", 100, HoodieJavaWriteClient::insert, - false, false, 100, 100, 1, Option.empty()); + false, false, 100, 100, 1, Option.empty(), INSTANT_GENERATOR); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, HoodieJavaWriteClient::upsert, - false, false, 5, 100, 2, config.populateMetaFields()); + false, false, 5, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule and execute compaction. Option timeStamp = client.scheduleCompaction(Option.empty()); @@ -125,14 +127,14 @@ public void testAsyncCompactionOnMORTable() throws Exception { // Insert String commitTime = client.createNewInstantTime(); insertBatch(config, client, commitTime, "000", 100, HoodieJavaWriteClient::insert, - false, false, 100, 100, 1, Option.empty()); + false, false, 100, 100, 1, Option.empty(), INSTANT_GENERATOR); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, HoodieJavaWriteClient::upsert, - false, false, 5, 100, 2, config.populateMetaFields()); + false, false, 5, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule compaction but do not run it Option timeStamp = client.scheduleCompaction(Option.empty()); @@ -142,7 +144,7 @@ public void testAsyncCompactionOnMORTable() throws Exception { commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, HoodieJavaWriteClient::upsert, - false, false, 5, 150, 2, config.populateMetaFields()); + false, false, 5, 150, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Verify all the records. metaClient.reloadActiveTimeline(); assertDataInMORTable(config, commitTime, timeStamp.get(), storageConf, Arrays.asList(dataGen.getPartitionPaths())); @@ -157,7 +159,7 @@ public void testAsyncCompactionOnMORTable() throws Exception { commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, HoodieJavaWriteClient::upsert, - false, false, 5, 200, 2, config.populateMetaFields()); + false, false, 5, 200, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Verify all the records. metaClient.reloadActiveTimeline(); assertDataInMORTable(config, commitTime, timeStamp.get(), storageConf, Arrays.asList(dataGen.getPartitionPaths())); diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java index eeebd7960a34..4714a9514c7e 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/HoodieJavaClientTestHarness.java @@ -41,9 +41,11 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -104,6 +106,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; @@ -460,8 +464,9 @@ public HoodieJavaTable getHoodieTable(HoodieTableMetaClient metaClient, HoodieWr public List insertFirstBatch(HoodieWriteConfig writeConfig, HoodieJavaWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit) throws Exception { - return insertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, isPreppedAPI, assertForCommit, expRecordsInThisCommit, true); + boolean assertForCommit, int expRecordsInThisCommit, InstantGenerator instantGenerator) throws Exception { + return insertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, isPreppedAPI, assertForCommit, + expRecordsInThisCommit, true, instantGenerator); } /** @@ -482,12 +487,12 @@ public List insertFirstBatch(HoodieWriteConfig writeConfig, HoodieJ public List insertFirstBatch(HoodieWriteConfig writeConfig, HoodieJavaWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false, filterForCommitTimeWithAssert); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false, filterForCommitTimeWithAssert, instantGenerator); } /** @@ -510,7 +515,8 @@ public List insertFirstBatch(HoodieWriteConfig writeConfig, HoodieJ public List insertBatch(HoodieWriteConfig writeConfig, HoodieJavaWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, Option partition) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + Option partition, InstantGenerator instantGenerator) throws Exception { if (partition.isPresent()) { final Function3, String, Integer, String> recordGenFunction = @@ -518,13 +524,13 @@ public List insertBatch(HoodieWriteConfig writeConfig, HoodieJavaWr return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, - partition.get()); + partition.get(), instantGenerator); } else { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, instantGenerator); } } @@ -532,9 +538,10 @@ public List updateBatch(HoodieWriteConfig writeConfig, HoodieJavaWr String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + InstantGenerator instantGenerator) throws Exception { return updateBatch(writeConfig, client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, writeFn, - isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, true); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, true, instantGenerator); } /** @@ -561,20 +568,20 @@ public List updateBatch(HoodieWriteConfig writeConfig, HoodieJavaWr int numRecordsInThisCommit, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, false, filterForCommitTimeWithAssert); + expTotalCommits, false, filterForCommitTimeWithAssert, instantGenerator); } public List deleteBatch(HoodieWriteConfig writeConfig, HoodieJavaWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, boolean assertForCommit, - int expRecordsInThisCommit, int expTotalRecords) throws Exception { + int expRecordsInThisCommit, int expTotalRecords, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return deleteBatch(writeConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, isPreppedAPI, - assertForCommit, expRecordsInThisCommit, expTotalRecords, true); + assertForCommit, expRecordsInThisCommit, expTotalRecords, true, timelineFactory, instantGenerator); } /** @@ -595,7 +602,8 @@ public List deleteBatch(HoodieWriteConfig writeConfig, HoodieJavaWr */ public List deleteBatch(HoodieWriteConfig writeConfig, HoodieJavaWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert, + TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { if (isPreppedAPI) { final Function2, String, Integer> recordGenFunction = @@ -607,7 +615,8 @@ public List deleteBatch(HoodieWriteConfig writeConfig, HoodieJavaWr Function3, HoodieJavaWriteClient, List, String> deleteFn = HoodieJavaWriteClient::deletePrepped; List result = deleteFn.apply(client, deleteRecords, newCommitTime); - return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, result); + return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, + result, timelineFactory, instantGenerator); } else { final Function> keyGenFunction = generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes); @@ -622,7 +631,8 @@ public List deleteBatch(HoodieWriteConfig writeConfig, HoodieJavaWr Function3, HoodieJavaWriteClient, List, String> deleteFn = HoodieJavaWriteClient::delete; List result = deleteFn.apply(client, deleteRecords, newCommitTime); - return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, result); + return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, + filterForCommitTimeWithAssert, result, timelineFactory, instantGenerator); } } @@ -630,9 +640,10 @@ public List writeBatch(HoodieJavaWriteClient client, String newComm Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, Function3, HoodieJavaWriteClient, List, String> writeFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + boolean doCommit, InstantGenerator instantGenerator) throws Exception { return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, - writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, instantGenerator); } public List writeBatch(HoodieJavaWriteClient client, String newCommitTime, String prevCommitTime, @@ -640,9 +651,9 @@ public List writeBatch(HoodieJavaWriteClient client, String newComm Function3, String, Integer, String> recordGenFunction, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean doCommit, String partition) throws Exception { + boolean doCommit, String partition, InstantGenerator instantGenerator) throws Exception { return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, - writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition); + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition, instantGenerator); } /** @@ -668,12 +679,12 @@ public List writeBatch(HoodieJavaWriteClient client, String newComm Function2, String, Integer> recordGenFunction, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, doCommit, filterForCommitTimeWithAssert); + expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } public List writeBatch(HoodieJavaWriteClient client, String newCommitTime, String prevCommitTime, @@ -682,12 +693,12 @@ public List writeBatch(HoodieJavaWriteClient client, String newComm Function3, HoodieJavaWriteClient, List, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert, - String partition) throws Exception { + String partition, InstantGenerator instantGenerator) throws Exception { List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit, partition); return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, doCommit, filterForCommitTimeWithAssert); + expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } private List writeBatchHelper(HoodieJavaWriteClient client, String newCommitTime, String prevCommitTime, @@ -695,7 +706,7 @@ private List writeBatchHelper(HoodieJavaWriteClient client, String int numRecordsInThisCommit, List records, Function3, HoodieJavaWriteClient, List, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, - int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws IOException { + int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws IOException { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); @@ -715,10 +726,10 @@ private List writeBatchHelper(HoodieJavaWriteClient client, String if (assertForCommit) { assertEquals(expTotalCommits, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(), "Expecting " + expTotalCommits + " commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be " + newCommitTime); if (filterForCommitTimeWithAssert) { // when meta cols are disabled, we can't really do per commit assertion. - assertEquals(expRecordsInThisCommit, numRowsInCommit(basePath, timeline, newCommitTime, true), + assertEquals(expRecordsInThisCommit, numRowsInCommit(basePath, timeline, newCommitTime, true, instantGenerator), "Must contain " + expRecordsInThisCommit + " records"); } @@ -732,12 +743,12 @@ private List writeBatchHelper(HoodieJavaWriteClient client, String if (filterForCommitTimeWithAssert) { // Check that the incremental consumption from prevCommitTime - assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true), + assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true, instantGenerator), countRecordsOptionallySince(basePath, timeline, Option.of(prevCommitTime)), "Incremental consumption from " + prevCommitTime + " should give all records in latest commit"); if (commitTimesBetweenPrevAndNew.isPresent()) { commitTimesBetweenPrevAndNew.get().forEach(ct -> { - assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true), + assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true, instantGenerator), countRecordsOptionallySince(basePath, timeline, Option.of(ct)), "Incremental consumption from " + ct + " should give all records in latest commit"); }); @@ -881,20 +892,21 @@ public static List tagLocation( } private List getWriteStatusAndVerifyDeleteOperation(String newCommitTime, String prevCommitTime, String initCommitTime, boolean assertForCommit, int expRecordsInThisCommit, - int expTotalRecords, boolean filerForCommitTimeWithAssert, List result) { + int expTotalRecords, boolean filerForCommitTimeWithAssert, List result, + TimelineFactory timelineFactory, InstantGenerator instantGenerator) { assertNoWriteErrors(result); // verify that there is a commit HoodieTableMetaClient metaClient = createMetaClient(); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + HoodieTimeline timeline = timelineFactory.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); if (assertForCommit) { assertEquals(3, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(), "Expecting 3 commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be " + newCommitTime); if (filerForCommitTimeWithAssert) { // if meta cols are disabled, we can't do assertion based on assertion time - assertEquals(expRecordsInThisCommit, numRowsInCommit(basePath, timeline, newCommitTime, true), + assertEquals(expRecordsInThisCommit, numRowsInCommit(basePath, timeline, newCommitTime, true, instantGenerator), "Must contain " + expRecordsInThisCommit + " records"); } @@ -908,7 +920,7 @@ private List getWriteStatusAndVerifyDeleteOperation(String newCommi if (filerForCommitTimeWithAssert) { // Check that the incremental consumption from prevCommitTime - assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true), + assertEquals(numRowsInCommit(basePath, timeline, newCommitTime, true, instantGenerator), countRecordsOptionallySince(basePath, timeline, Option.of(prevCommitTime)), "Incremental consumption from " + prevCommitTime + " should give no records in latest commit," + " since it is a delete operation"); @@ -918,8 +930,8 @@ private List getWriteStatusAndVerifyDeleteOperation(String newCommi } public long numRowsInCommit(String basePath, HoodieTimeline commitTimeline, - String instantTime, boolean filterByCommitTime) { - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime); + String instantTime, boolean filterByCommitTime, InstantGenerator instantGenerator) { + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime); if (!commitTimeline.containsInstant(commitInstant)) { throw new HoodieException("No commit exists at " + instantTime); } @@ -945,9 +957,10 @@ public long numRowsInCommit(String basePath, HoodieTimeline commitTimeline, private static HashMap getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline, List commitsToReturn) throws IOException { HashMap fileIdToFullPath = new HashMap<>(); + TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); for (HoodieInstant commit : commitsToReturn) { HoodieCommitMetadata metadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(new StoragePath(basePath))); } return fileIdToFullPath; @@ -1009,7 +1022,7 @@ public long countRecordsOptionallySince(String basePath, HoodieTimeline commitTi } else if (paths[0].endsWith(HoodieFileFormat.HFILE.getFileExtension())) { Stream genericRecordStream = readHFile(context.getStorageConf(), paths); if (lastCommitTimeOpt.isPresent()) { - return genericRecordStream.filter(gr -> HoodieTimeline.compareTimestamps(lastCommitTimeOpt.get(), HoodieActiveTimeline.LESSER_THAN, + return genericRecordStream.filter(gr -> compareTimestamps(lastCommitTimeOpt.get(), LESSER_THAN, gr.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString())) .count(); } else { diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java index 98618bf55ca5..0e7f72cbd33d 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieJavaEngineContext; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; @@ -250,7 +251,7 @@ protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, WriteOper protected void archiveDataTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws IOException { HoodieTable table = HoodieJavaTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(writeConfig, table); archiver.archiveIfRequired(context); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java index 37e3d1d6cca6..72cabea11321 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkClusteringClient.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.spark.api.java.JavaRDD; @@ -47,16 +46,16 @@ public HoodieSparkClusteringClient( } @Override - public void cluster(HoodieInstant instant) throws IOException { - LOG.info("Executing clustering instance " + instant); + public void cluster(String instantTime) throws IOException { + LOG.info("Executing clustering instance " + instantTime); SparkRDDWriteClient writeClient = (SparkRDDWriteClient) clusteringClient; - Option commitMetadata = writeClient.cluster(instant.getTimestamp()).getCommitMetadata(); + Option commitMetadata = writeClient.cluster(instantTime).getCommitMetadata(); Stream hoodieWriteStatStream = commitMetadata.get().getPartitionToWriteStats().entrySet().stream().flatMap(e -> e.getValue().stream()); long errorsCount = hoodieWriteStatStream.mapToLong(HoodieWriteStat::getTotalWriteErrors).sum(); if (errorsCount > 0) { // TODO: Should we treat this fatal and throw exception? - LOG.error("Clustering for instant (" + instant + ") failed with write errors"); + LOG.error("Clustering for instant (" + instantTime + ") failed with write errors"); } } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java index ce655bdffb04..a7017c726ae1 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/HoodieSparkCompactor.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.table.action.HoodieWriteMetadata; @@ -45,19 +44,19 @@ public HoodieSparkCompactor(BaseHoodieWriteClient>, J } @Override - public void compact(HoodieInstant instant) { - LOG.info("Compactor executing compaction " + instant); + public void compact(String instantTime) { + LOG.info("Compactor executing compaction " + instantTime); SparkRDDWriteClient writeClient = (SparkRDDWriteClient) compactionClient; - HoodieWriteMetadata> compactionMetadata = writeClient.compact(instant.getTimestamp()); + HoodieWriteMetadata> compactionMetadata = writeClient.compact(instantTime); List writeStats = compactionMetadata.getCommitMetadata().get().getWriteStats(); long numWriteErrors = writeStats.stream().mapToLong(HoodieWriteStat::getTotalWriteErrors).sum(); if (numWriteErrors != 0) { // We treat even a single error in compaction as fatal - LOG.error("Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors); + LOG.error("Compaction for instant (" + instantTime + ") failed with write errors. Errors :" + numWriteErrors); throw new HoodieException( - "Compaction for instant (" + instant + ") failed with write errors. Errors :" + numWriteErrors); + "Compaction for instant (" + instantTime + ") failed with write errors. Errors :" + numWriteErrors); } // Commit compaction - writeClient.commitCompaction(instant.getTimestamp(), compactionMetadata.getCommitMetadata().get(), Option.empty()); + writeClient.commitCompaction(instantTime, compactionMetadata.getCommitMetadata().get(), Option.empty()); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java index 7cb42f9182a6..7c07468c43c8 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDReadClient.java @@ -228,7 +228,7 @@ public List> getPendingCompactions() { .setConf(storageConf.newInstance()).setBasePath(hoodieTable.getMetaClient().getBasePath()).setLoadActiveTimelineOnLoad(true).build(); return CompactionUtils.getAllPendingCompactionPlans(metaClient).stream() .map( - instantWorkloadPair -> Pair.of(instantWorkloadPair.getKey().getTimestamp(), instantWorkloadPair.getValue())) + instantWorkloadPair -> Pair.of(instantWorkloadPair.getKey().requestedTime(), instantWorkloadPair.getValue())) .collect(Collectors.toList()); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java index 847ab48115e4..2f313b1ddd72 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDTableServiceClient.java @@ -49,7 +49,7 @@ protected SparkRDDTableServiceClient(HoodieEngineContext context, protected void validateClusteringCommit(HoodieWriteMetadata> clusteringMetadata, String clusteringCommitTime, HoodieTable table) { if (clusteringMetadata.getWriteStatuses().isEmpty()) { HoodieClusteringPlan clusteringPlan = ClusteringUtils.getClusteringPlan( - table.getMetaClient(), ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline()).get()) + table.getMetaClient(), ClusteringUtils.getInflightClusteringInstant(clusteringCommitTime, table.getActiveTimeline(), table.getInstantGenerator()).get()) .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException( "Unable to read clustering plan for instant: " + clusteringCommitTime)); throw new HoodieClusteringException("Clustering plan produced 0 WriteStatus for " + clusteringCommitTime diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java index 4d7eda5be20f..4a9c326b2962 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/utils/SparkMetadataWriterUtils.java @@ -185,7 +185,7 @@ private static List getUnmergedLogFileRecords(List logFile .withBasePath(metaClient.getBasePath()) .withLogFilePaths(logFilePaths) .withBufferSize(MAX_DFS_STREAM_BUFFER_SIZE.defaultValue()) - .withLatestInstantTime(metaClient.getActiveTimeline().getCommitsTimeline().lastInstant().get().getTimestamp()) + .withLatestInstantTime(metaClient.getActiveTimeline().getCommitsTimeline().lastInstant().get().requestedTime()) .withReaderSchema(readerSchema) .withTableMetaClient(metaClient) .withLogRecordScannerCallback(records::add) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java index 1bd743935407..3fc6fe83310d 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/bucket/HoodieSparkConsistentBucketIndex.java @@ -64,7 +64,7 @@ public HoodieData updateLocation(HoodieData writeStatu String instantTime) throws HoodieIndexException { HoodieInstant instant = hoodieTable.getMetaClient().getActiveTimeline().findInstantsAfterOrEquals(instantTime, 1).firstInstant().get(); - ValidationUtils.checkState(instant.getTimestamp().equals(instantTime), "Cannot get the same instant, instantTime: " + instantTime); + ValidationUtils.checkState(instant.requestedTime().equals(instantTime), "Cannot get the same instant, instantTime: " + instantTime); if (!ClusteringUtils.isClusteringOrReplaceCommitAction(instant.getAction())) { return writeStatuses; } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java index acbdbd2413c0..3662e5da880f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/index/hbase/SparkHoodieHBaseIndex.java @@ -31,8 +31,8 @@ import org.apache.hudi.common.model.HoodieRecordLocation; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.RateLimiter; import org.apache.hudi.common.util.ReflectionUtils; @@ -603,7 +603,7 @@ public boolean rollbackCommit(String instantTime) { final RateLimiter limiter = RateLimiter.create(multiPutBatchSize, TimeUnit.SECONDS); try (HTable hTable = (HTable) hbaseConnection.getTable(TableName.valueOf(tableName)); BufferedMutator mutator = hbaseConnection.getBufferedMutator(TableName.valueOf(tableName))) { - Long rollbackTime = HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(); + Long rollbackTime = TimelineUtils.parseDateFromInstantTime(instantTime).getTime(); Long currentTime = new Date().getTime(); Scan scan = new Scan(); scan.addFamily(SYSTEM_COLUMN_FAMILY); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java index 0dac23e65057..0ff6d8ab551f 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/HoodieSparkTable.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieException; @@ -61,7 +60,6 @@ public static HoodieSparkTable create(HoodieWriteConfig config, HoodieEng .setConf(context.getStorageConf().newInstance()) .setBasePath(config.getBasePath()) .setLoadActiveTimelineOnLoad(true).setConsistencyGuardConfig(config.getConsistencyGuardConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(config.getTimelineLayoutVersion()))) .setTimeGeneratorConfig(config.getTimeGeneratorConfig()) .setFileSystemRetryConfig(config.getFileSystemRetryConfig()) .setMetaserverConfig(config.getProps()).build(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java index b19a202cc77d..e7175c55d9f6 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/bootstrap/SparkBootstrapCommitActionExecutor.java @@ -150,9 +150,9 @@ protected Option>> metadataBootstrap HoodieTableMetaClient metaClient = table.getMetaClient(); String bootstrapInstantTime = HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(), bootstrapInstantTime)); + instantGenerator.createNewInstant(State.REQUESTED, metaClient.getCommitActionType(), bootstrapInstantTime)); - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + table.getActiveTimeline().transitionRequestedToInflight(instantGenerator.createNewInstant(State.REQUESTED, metaClient.getCommitActionType(), bootstrapInstantTime), Option.empty()); HoodieData bootstrapWriteStatuses = runMetadataBootstrap(partitionFilesList); @@ -237,7 +237,7 @@ protected Option>> fullBootstrap(Lis partitionFilesList, config); // Start Full Bootstrap String bootstrapInstantTime = HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS; - final HoodieInstant requested = new HoodieInstant( + final HoodieInstant requested = instantGenerator.createNewInstant( State.REQUESTED, table.getMetaClient().getCommitActionType(), bootstrapInstantTime); table.getActiveTimeline().createNewInstant(requested); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java index 288b0d24de9a..c6d0cf9ac972 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/cluster/SparkExecuteClusteringCommitActionExecutor.java @@ -42,7 +42,7 @@ public SparkExecuteClusteringCommitActionExecutor(HoodieEngineContext context, String instantTime) { super(context, config, table, instantTime, WriteOperationType.CLUSTER); this.clusteringPlan = ClusteringUtils.getClusteringPlan( - table.getMetaClient(), ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline()).get()) + table.getMetaClient(), ClusteringUtils.getRequestedClusteringInstant(instantTime, table.getActiveTimeline(), table.getInstantGenerator()).get()) .map(Pair::getRight).orElseThrow(() -> new HoodieClusteringException( "Unable to read clustering plan for instant: " + instantTime)); } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/AverageRecordSizeUtils.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/AverageRecordSizeUtils.java index b0cd1ca5f7ee..8873fb01c97b 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/AverageRecordSizeUtils.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/AverageRecordSizeUtils.java @@ -21,6 +21,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.config.HoodieWriteConfig; @@ -46,7 +47,7 @@ public class AverageRecordSizeUtils { * Obtains the average record size based on records written during previous commits. Used for estimating how many * records pack into one file. */ - static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig) { + static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConfig hoodieWriteConfig, CommitMetadataSerDe commitMetadataSerDe) { long avgSize = hoodieWriteConfig.getCopyOnWriteRecordSizeEstimate(); long fileSizeThreshold = (long) (hoodieWriteConfig.getRecordSizeEstimationThreshold() * hoodieWriteConfig.getParquetSmallFileLimit()); if (!commitTimeline.empty()) { @@ -55,8 +56,8 @@ static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConf while (instants.hasNext()) { HoodieInstant instant = instants.next(); try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = commitMetadataSerDe + .deserialize(instant, commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); if (instant.getAction().equals(COMMIT_ACTION) || instant.getAction().equals(REPLACE_COMMIT_ACTION)) { long totalBytesWritten = commitMetadata.fetchTotalBytesWritten(); long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten(); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java index 2f57f6bb18b6..7c75ea796b5e 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkBulkInsertHelper.java @@ -70,7 +70,7 @@ public HoodieWriteMetadata> bulkInsert(final HoodieData< HoodieWriteMetadata result = new HoodieWriteMetadata(); // Transition bulk_insert state to inflight - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, + table.getActiveTimeline().transitionRequestedToInflight(table.getInstantGenerator().createNewInstant(HoodieInstant.State.REQUESTED, executor.getCommitActionType(), instantTime), Option.empty(), config.shouldAllowMultiWriteOnSameInstant()); diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java index 843581a36a05..a38721c66b92 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/SparkDeletePartitionCommitActionExecutor.java @@ -76,10 +76,10 @@ public HoodieWriteMetadata> execute() { // created requested HoodieInstant dropPartitionsInstant = - new HoodieInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime); + instantGenerator.createNewInstant(REQUESTED, REPLACE_COMMIT_ACTION, instantTime); if (!table.getStorage().exists( new StoragePath(table.getMetaClient().getMetaPath(), - dropPartitionsInstant.getFileName()))) { + instantFileNameGenerator.getFileName(dropPartitionsInstant)))) { HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setOperationType(WriteOperationType.DELETE_PARTITION.name()) diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java index 6536506adbda..0f4bcef2fb13 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/commit/UpsertPartitioner.java @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; @@ -176,9 +177,10 @@ private void assignInserts(WorkloadProfile profile, HoodieEngineContext context) * created by clustering, which has smaller average record size, which affects assigning inserts and * may result in OOM by making spark underestimate the actual input record sizes. */ + TimelineLayout layout = TimelineLayout.fromVersion(table.getActiveTimeline().getTimelineLayoutVersion()); long averageRecordSize = AverageRecordSizeUtils.averageBytesPerRecord(table.getMetaClient().getActiveTimeline() .getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION)) - .filterCompletedInstants(), config); + .filterCompletedInstants(), config, layout.getCommitMetadataSerDe()); LOG.info("AvgRecordSize => " + averageRecordSize); Map> partitionSmallFilesMap = @@ -311,7 +313,7 @@ protected List getSmallFiles(String partitionPath) { if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); List allFiles = table.getBaseFileOnlyView() - .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.requestedTime()).collect(Collectors.toList()); for (HoodieBaseFile file : allFiles) { if (file.getFileSize() < config.getParquetSmallFileLimit()) { diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java index cd9bf334f635..d47eb6d33aa9 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/compact/HoodieSparkMergeOnReadTableCompactor.java @@ -45,11 +45,11 @@ public class HoodieSparkMergeOnReadTableCompactor public void preCompact( HoodieTable table, HoodieTimeline pendingCompactionTimeline, WriteOperationType operationType, String instantTime) { HoodieInstant requestedCompactionInstantTime = WriteOperationType.COMPACT.equals(operationType) - ? HoodieTimeline.getCompactionRequestedInstant(instantTime) - : HoodieTimeline.getLogCompactionRequestedInstant(instantTime); + ? table.getInstantGenerator().getCompactionRequestedInstant(instantTime) + : table.getInstantGenerator().getLogCompactionRequestedInstant(instantTime); if (!pendingCompactionTimeline.containsInstant(requestedCompactionInstantTime)) { throw new IllegalStateException( - "No Compaction request available at " + requestedCompactionInstantTime.getTimestamp() + " to run compaction"); + "No Compaction request available at " + requestedCompactionInstantTime.requestedTime() + " to run compaction"); } } diff --git a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java index ac2cf3203563..9f6a78f63679 100644 --- a/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java +++ b/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/table/action/deltacommit/SparkUpsertDeltaCommitPartitioner.java @@ -92,7 +92,7 @@ private List getSmallFileCandidates(String partitionPath, HoodieInsta // pending compaction if (table.getIndex().canIndexLogFiles()) { return table.getSliceView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false) + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.requestedTime(), false) .filter(this::isSmallFile) .collect(Collectors.toList()); } @@ -104,7 +104,7 @@ private List getSmallFileCandidates(String partitionPath, HoodieInsta // If we cannot index log files, then we choose the smallest parquet file in the partition and add inserts to // it. Doing this overtime for a partition, we ensure that we handle small file issues return table.getSliceView() - .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.getTimestamp(), false) + .getLatestFileSlicesBeforeOrOn(partitionPath, latestCommitInstant.requestedTime(), false) .filter( fileSlice -> // NOTE: We can not pad slices with existing log-files w/o compacting these, 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 fc89bbec981d..b8d9c300a2f1 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 @@ -66,6 +66,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.util.StringUtils.EMPTY_STRING; import static org.apache.hudi.table.action.restore.RestoreUtils.getRestorePlan; import static org.apache.hudi.table.action.restore.RestoreUtils.getSavepointToRestoreTimestampV1Schema; @@ -176,7 +178,7 @@ public void testSavepointAndRollback(Boolean testFailedRestore, Boolean failedRe // rollback to savepoint 002 HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstantsAsStream().findFirst().get(); - client.restoreToSavepoint(savepoint.getTimestamp()); + client.restoreToSavepoint(savepoint.requestedTime()); metaClient = HoodieTableMetaClient.reload(metaClient); table = HoodieSparkTable.create(getConfig(), context, metaClient); @@ -193,19 +195,19 @@ public void testSavepointAndRollback(Boolean testFailedRestore, Boolean failedRe if (testFailedRestore) { //test to make sure that restore commit is reused when the restore fails and is re-ran HoodieInstant inst = table.getActiveTimeline().getRestoreTimeline().getInstants().get(0); - String restoreFileName = table.getMetaClient().getBasePath() + "/.hoodie/" + inst.getFileName(); + String restoreFileName = table.getMetaClient().getBasePath() + "/.hoodie/" + INSTANT_FILE_NAME_GENERATOR.getFileName(inst); //delete restore commit file assertTrue((new File(restoreFileName)).delete()); if (!failedRestoreInflight) { //delete restore inflight file - HoodieInstant inflightInst = new HoodieInstant(true, inst.getAction(), inst.getTimestamp()); - assertTrue((new File(table.getMetaClient().getBasePath() + "/.hoodie/" + inflightInst.getFileName())).delete()); + HoodieInstant inflightInst = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, inst.getAction(), inst.requestedTime()); + assertTrue((new File(table.getMetaClient().getBasePath() + "/.hoodie/" + INSTANT_FILE_NAME_GENERATOR.getFileName(inflightInst))).delete()); } try (SparkRDDWriteClient newClient = getHoodieWriteClient(cfg)) { //restore again - newClient.restoreToSavepoint(savepoint.getTimestamp()); + newClient.restoreToSavepoint(savepoint.requestedTime()); //verify that we reuse the existing restore commit metaClient = HoodieTableMetaClient.reload(metaClient); @@ -213,7 +215,7 @@ public void testSavepointAndRollback(Boolean testFailedRestore, Boolean failedRe List restoreInstants = table.getActiveTimeline().getRestoreTimeline().getInstants(); assertEquals(1, restoreInstants.size()); assertEquals(HoodieInstant.State.COMPLETED, restoreInstants.get(0).getState()); - assertEquals(inst.getTimestamp(), restoreInstants.get(0).getTimestamp()); + assertEquals(inst.requestedTime(), restoreInstants.get(0).requestedTime()); } } } @@ -260,7 +262,7 @@ public void testGetSavepointOldSchema() throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieSparkTable table = HoodieSparkTable.create(getConfig(), context, metaClient); HoodieInstant savepoint = table.getCompletedSavepointTimeline().lastInstant().get(); - client.restoreToSavepoint(savepoint.getTimestamp()); + client.restoreToSavepoint(savepoint.requestedTime()); //verify that getSavepointToRestoreTimestampV1Schema is correct metaClient = HoodieTableMetaClient.reload(metaClient); @@ -353,7 +355,7 @@ public void testSavepointAndRollbackWithKeepLatestFileVersionPolicy() throws Exc // rollback to savepoint 002 HoodieInstant savepoint = table.getCompletedSavepointTimeline().getInstantsAsStream().findFirst().get(); - client.restoreToSavepoint(savepoint.getTimestamp()); + client.restoreToSavepoint(savepoint.requestedTime()); metaClient = HoodieTableMetaClient.reload(metaClient); table = HoodieSparkTable.create(getConfig(), context, metaClient); @@ -552,7 +554,7 @@ public void testFailedRollbackCommit( HoodieInstant rollbackInstant = rollbackInstants.get(0); // delete rollback completed meta file and retry rollback. - FileCreateUtils.deleteRollbackCommit(basePath, rollbackInstant.getTimestamp()); + FileCreateUtils.deleteRollbackCommit(basePath, rollbackInstant.requestedTime()); if (instantToRollbackExists) { // recreate actual commit files if needed @@ -777,10 +779,10 @@ public void testRollbackWithRequestedRollbackPlan(boolean enableMetadataTable, b if (isRollbackPlanCorrupted) { // Should create a new rollback instant - assertNotEquals(rollbackInstantTime, rollbackInstant.getTimestamp()); + assertNotEquals(rollbackInstantTime, rollbackInstant.requestedTime()); } else { // Should reuse the rollback instant - assertEquals(rollbackInstantTime, rollbackInstant.getTimestamp()); + assertEquals(rollbackInstantTime, rollbackInstant.requestedTime()); } } if (metadataWriter != null) { 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 ba6fb446a893..8061066bd4b5 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 @@ -100,6 +100,7 @@ import static org.apache.hudi.common.config.LockConfiguration.ZK_CONNECT_URL_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.ZK_LOCK_KEY_PROP_KEY; import static org.apache.hudi.common.config.LockConfiguration.ZK_SESSION_TIMEOUT_MS_PROP_KEY; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -281,7 +282,7 @@ private void testHoodieClientBasicMultiWriterWithEarlyConflictDetection(String t List completedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline() .filterCompletedInstants().getInstants().stream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .map(HoodieInstant::requestedTime).collect(Collectors.toList()); assertEquals(3, completedInstant.size()); assertTrue(completedInstant.contains(nextCommitTime1)); @@ -583,12 +584,12 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta String pendingCompactionTime = (tableType == HoodieTableType.MERGE_ON_READ) ? metaClient.reloadActiveTimeline().filterPendingCompactionTimeline() - .firstInstant().get().getTimestamp() + .firstInstant().get().requestedTime() : ""; Option pendingCleanInstantOp = metaClient.reloadActiveTimeline().getCleanerTimeline().filterInflightsAndRequested() .firstInstant(); String pendingCleanTime = pendingCleanInstantOp.isPresent() - ? pendingCleanInstantOp.get().getTimestamp() + ? pendingCleanInstantOp.get().requestedTime() : client.createNewInstantTime(); CountDownLatch runCountDownLatch = new CountDownLatch(threadCount); @@ -627,9 +628,9 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta validInstants.addAll( metaClient.reloadActiveTimeline().getCompletedReplaceTimeline() - .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet())); + .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet())); Set completedInstants = metaClient.reloadActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp) + .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::requestedTime) .collect(Collectors.toSet()); assertTrue(validInstants.containsAll(completedInstants)); @@ -764,13 +765,13 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) String commitTimeBetweenPrevAndNew = "002"; JavaRDD result1 = updateBatch(cfg, client1, newCommitTime, "001", Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); // Start and finish another commit while the previous writer for commit 003 is running newCommitTime = "004"; SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); JavaRDD result2 = updateBatch(cfg2, client2, newCommitTime, "001", Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); client2.commit(newCommitTime, result2); // Schedule and run clustering while previous writer for commit 003 is running SparkRDDWriteClient client3 = getHoodieWriteClient(cfg3); @@ -1022,7 +1023,7 @@ private void createCommitWithInsertsForPartition(HoodieWriteConfig cfg, SparkRDD String prevCommitTime, String newCommitTime, int numRecords, String partition) throws Exception { JavaRDD result = insertBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::insert, - false, false, numRecords, numRecords, 1, Option.of(partition)); + false, false, numRecords, numRecords, 1, Option.of(partition), INSTANT_GENERATOR); assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); } @@ -1031,7 +1032,7 @@ private JavaRDD createCommitWithInserts(HoodieWriteConfig cfg, Spar boolean doCommit) throws Exception { // Finish first base commit JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::bulkInsert, - false, false, numRecords); + false, false, numRecords, INSTANT_GENERATOR); if (doCommit) { assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); } @@ -1043,7 +1044,7 @@ private void createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient throws Exception { JavaRDD result = updateBatch(cfg, client, newCommitTime, prevCommit, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); client.commit(newCommitTime, result); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java index 9a3fc6095031..b1a6d17acece 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestHoodieReadClient.java @@ -38,6 +38,7 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -205,7 +206,7 @@ private void testTagLocation(HoodieWriteConfig hoodieWriteConfig, String initCommitTime = "000"; int numRecords = 200; JavaRDD result = insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, - numRecords, insertFn, isPrepped, true, numRecords); + numRecords, insertFn, isPrepped, true, numRecords, INSTANT_GENERATOR); // Construct HoodieRecord from the WriteStatus but set HoodieKey, Data and HoodieRecordLocation accordingly // since they have been modified in the DAG JavaRDD recordRDD = @@ -223,7 +224,7 @@ private void testTagLocation(HoodieWriteConfig hoodieWriteConfig, String commitTimeBetweenPrevAndNew = "002"; result = updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, updateFn, isPrepped, true, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); recordRDD = jsc.parallelize(result.collect().stream().map(WriteStatus::getWrittenRecordDelegates).flatMap(Collection::stream) .map(recordDelegate -> new HoodieAvroRecord(recordDelegate.getHoodieKey(), null)).collect(Collectors.toList()), PARALLELISM); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java index 0dd34ca3964e..adff56e9d495 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiFS.java @@ -22,7 +22,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -53,6 +52,8 @@ import java.io.IOException; import java.util.List; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.junit.jupiter.api.Assertions.assertEquals; public class TestMultiFS extends HoodieSparkClientTestHarness { @@ -135,8 +136,8 @@ public void readLocalWriteHDFS() throws Exception { // Read from hdfs FileSystem fs = HadoopFSUtils.getFs(dfsBasePath, HoodieTestUtils.getDefaultStorageConf()); HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(HadoopFSUtils.getStorageConf(fs.getConf()), dfsBasePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); - Dataset readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime); + HoodieTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + Dataset readRecords = HoodieClientTestUtils.readCommit(dfsBasePath, sqlContext, timeline, readCommitTime, true, INSTANT_GENERATOR); assertEquals(readRecords.count(), records.size()); // Write to local @@ -156,9 +157,9 @@ public void readLocalWriteHDFS() throws Exception { LOG.info("Reading from path: " + tablePath); fs = HadoopFSUtils.getFs(tablePath, HoodieTestUtils.getDefaultStorageConf()); metaClient = HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(fs.getConf()), tablePath); - timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); Dataset localReadRecords = - HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime); + HoodieClientTestUtils.readCommit(tablePath, sqlContext, timeline, writeCommitTime, true, INSTANT_GENERATOR); assertEquals(localReadRecords.count(), localRecords.size()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiWriterWithPreferWriterIngestion.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiWriterWithPreferWriterIngestion.java index 007be3e9cb13..53611917722a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiWriterWithPreferWriterIngestion.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestMultiWriterWithPreferWriterIngestion.java @@ -58,6 +58,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.config.LockConfiguration.FILESYSTEM_LOCK_PATH_PROP_KEY; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -189,7 +190,7 @@ public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType ta future2.get(); future3.get(); Set completedInstants = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp) + .filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::requestedTime) .collect(Collectors.toSet()); Assertions.assertTrue(validInstants.containsAll(completedInstants)); } @@ -222,13 +223,13 @@ public void testHoodieClientMultiWriterWithClustering(HoodieTableType tableType) JavaRDD result1 = updateBatch(cfg, client1, instant2, instant1, Option.of(Arrays.asList(instant1)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); // Start and finish another commit while the previous writer for commit 003 is running String instant3 = client1.createNewInstantTime(); SparkRDDWriteClient client2 = getHoodieWriteClient(cfg); JavaRDD result2 = updateBatch(cfg, client2, instant3, instant1, Option.of(Arrays.asList(instant1)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); client2.commit(instant3, result2); // Schedule and run clustering while previous writer for commit 003 is running SparkRDDWriteClient client3 = getHoodieWriteClient(cfg); @@ -243,7 +244,7 @@ private void createCommitWithInserts(HoodieWriteConfig cfg, SparkRDDWriteClient String prevCommitTime, String newCommitTime, int numRecords) throws Exception { // Finish first base commmit JavaRDD result = insertFirstBatch(cfg, client, newCommitTime, prevCommitTime, numRecords, SparkRDDWriteClient::bulkInsert, - false, false, numRecords); + false, false, numRecords, INSTANT_GENERATOR); assertTrue(client.commit(newCommitTime, result), "Commit should succeed"); } @@ -252,7 +253,7 @@ private void createCommitWithUpserts(HoodieWriteConfig cfg, SparkRDDWriteClient throws Exception { JavaRDD result = updateBatch(cfg, client, newCommitTime, prevCommit, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", numRecords, SparkRDDWriteClient::upsert, false, false, - numRecords, 200, 2); + numRecords, 200, 2, INSTANT_GENERATOR); client.commit(newCommitTime, result); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java index bea66a7fefda..7163df31ad69 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java @@ -111,7 +111,8 @@ public void testSavepoint(boolean enableMetadataTable, .getPartitionMetadata(); HoodieTimeline commitsTimeline = table.getActiveTimeline().getCommitsTimeline(); - Map> partitionToWriteStats = HoodieCommitMetadata.fromBytes( + Map> partitionToWriteStats = metaClient.getCommitMetadataSerDe().deserialize( + commitsTimeline.lastInstant().get(), commitsTimeline.getInstantDetails(commitsTimeline.lastInstant().get()).get(), HoodieCommitMetadata.class) .getPartitionToWriteStats(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java index 40aad26c45dc..5b1d52c5a66d 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestTableSchemaEvolution.java @@ -48,7 +48,6 @@ import java.util.List; import java.util.stream.Collectors; -import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_1; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.EXTRA_TYPE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.FARE_NESTED_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.HOODIE_IS_DELETED_SCHEMA; @@ -57,6 +56,8 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_PREFIX; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA_SUFFIX; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -166,7 +167,6 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) .setTableType(HoodieTableType.MERGE_ON_READ) - .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); HoodieWriteConfig hoodieWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA, shouldAllowDroppedColumns); @@ -175,7 +175,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime, - numRecords, SparkRDDWriteClient::insert, false, false, numRecords); + numRecords, SparkRDDWriteClient::insert, false, false, numRecords, INSTANT_GENERATOR); checkLatestDeltaCommit("001"); // Compact once so we can incrementally read later @@ -185,14 +185,15 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // Updates with same schema is allowed final int numUpdateRecords = 5; updateBatch(hoodieWriteConfig, client, "003", "002", Option.empty(), - initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, 0, 0, 0, INSTANT_GENERATOR); checkLatestDeltaCommit("003"); checkReadRecords("000", numRecords); // Delete with same schema is allowed final int numDeleteRecords = 2; numRecords -= numDeleteRecords; - deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords, false, false, 0, 0); + deleteBatch(hoodieWriteConfig, client, "004", "003", initCommitTime, numDeleteRecords, false, false, 0, 0, + TIMELINE_FACTORY, INSTANT_GENERATOR); checkLatestDeltaCommit("004"); checkReadRecords("000", numRecords); @@ -204,7 +205,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // with a evolved schema and insertBatch inserts records using the TRIP_EXAMPLE_SCHEMA. try { writeBatch(client, "005", "004", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, false, numRecords, 2 * numRecords, 5, false, INSTANT_GENERATOR); assertTrue(shouldAllowDroppedColumns); } catch (HoodieInsertException e) { assertFalse(shouldAllowDroppedColumns); @@ -213,7 +214,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // Update with evolved schema (column dropped) might be allowed depending on config set. updateBatch(hoodieDevolvedWriteConfig, client, "006", "005", Option.empty(), - initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 2 * numRecords, 0); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 2 * numRecords, 0, INSTANT_GENERATOR); // Insert with an evolved scheme is allowed HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); @@ -223,7 +224,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // with an evolved schema and insertBatch inserts records using the TRIP_EXAMPLE_SCHEMA. final List evolvedRecords = generateInsertsWithSchema("007", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); writeBatch(client, "007", "006", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, numRecords, 3 * numRecords, 7, false); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, false, numRecords, 3 * numRecords, 7, false, INSTANT_GENERATOR); // new commit checkLatestDeltaCommit("007"); @@ -232,7 +233,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { // Updates with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("008", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); writeBatch(client, "008", "007", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, numRecords, 4 * numRecords, 8, false); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, false, numRecords, 4 * numRecords, 8, false, INSTANT_GENERATOR); // new commit checkLatestDeltaCommit("008"); checkReadRecords("000", 4 * numRecords); @@ -241,7 +242,7 @@ public void testMORTable(boolean shouldAllowDroppedColumns) throws Exception { client = getHoodieWriteClient(hoodieWriteConfig); try { updateBatch(hoodieWriteConfig, client, "009", "008", Option.empty(), - initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9); + initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, false, numUpdateRecords, 4 * numRecords, 9, INSTANT_GENERATOR); assertTrue(shouldAllowDroppedColumns); } catch (HoodieUpsertException e) { assertFalse(shouldAllowDroppedColumns); @@ -254,7 +255,6 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep // Create the table HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) - .setTimelineLayoutVersion(VERSION_1) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); HoodieWriteConfig hoodieWriteConfig = getWriteConfigBuilder(TRIP_EXAMPLE_SCHEMA) @@ -266,20 +266,21 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep // Initial inserts with TRIP_EXAMPLE_SCHEMA int numRecords = 10; insertFirstBatch(hoodieWriteConfig, client, "001", initCommitTime, - numRecords, SparkRDDWriteClient::insert, false, true, numRecords); + numRecords, SparkRDDWriteClient::insert, false, true, numRecords, INSTANT_GENERATOR); checkReadRecords("000", numRecords); // Updates with same schema is allowed final int numUpdateRecords = 5; updateBatch(hoodieWriteConfig, client, "002", "001", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, - numUpdateRecords, numRecords, 2); + numUpdateRecords, numRecords, 2, INSTANT_GENERATOR); checkReadRecords("000", numRecords); // Delete with same schema is allowed final int numDeleteRecords = 2; numRecords -= numDeleteRecords; - deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords, false, true, 0, numRecords); + deleteBatch(hoodieWriteConfig, client, "003", "002", initCommitTime, numDeleteRecords, false, true, 0, + numRecords, TIMELINE_FACTORY, INSTANT_GENERATOR); checkReadRecords("000", numRecords); // Inserting records w/ new evolved schema (w/ tip column dropped) @@ -288,7 +289,7 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep final List failedRecords = generateInsertsWithSchema("004", numRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_DROPPED); try { writeBatch(client, "004", "003", Option.empty(), "003", numRecords, - (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false); + (String s, Integer a) -> failedRecords, SparkRDDWriteClient::insert, true, numRecords, numRecords * 2, 1, false, INSTANT_GENERATOR); assertTrue(shouldAllowDroppedColumns); } catch (HoodieInsertException e) { assertFalse(shouldAllowDroppedColumns); @@ -298,7 +299,7 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep // Updating records w/ new evolved schema updateBatch(hoodieDevolvedWriteConfig, client, "005", "004", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, - numUpdateRecords, 2 * numRecords, 5); + numUpdateRecords, 2 * numRecords, 5, INSTANT_GENERATOR); // Inserting with evolved schema is allowed HoodieWriteConfig hoodieEvolvedWriteConfig = getWriteConfig(TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED, shouldAllowDroppedColumns); @@ -307,17 +308,17 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep // We cannot use insertBatch directly here because we want to insert records // with a evolved schema. writeBatch(client, "006", "005", Option.empty(), initCommitTime, numRecords, - (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 3 * numRecords, 6, false); + (String s, Integer a) -> evolvedRecords, SparkRDDWriteClient::insert, true, numRecords, 3 * numRecords, 6, false, INSTANT_GENERATOR); // new commit HoodieTimeline curTimeline = metaClient.reloadActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants(); - assertTrue(curTimeline.lastInstant().get().getTimestamp().equals("006")); + assertTrue(curTimeline.lastInstant().get().requestedTime().equals("006")); checkReadRecords("000", 3 * numRecords); // Updating with evolved schema is allowed final List updateRecords = generateUpdatesWithSchema("007", numUpdateRecords, TRIP_EXAMPLE_SCHEMA_EVOLVED_COL_ADDED); writeBatch(client, "007", "006", Option.empty(), initCommitTime, - numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 3 * numRecords, 7, false); + numUpdateRecords, (String s, Integer a) -> updateRecords, SparkRDDWriteClient::upsert, true, numUpdateRecords, 3 * numRecords, 7, false, INSTANT_GENERATOR); checkReadRecords("000", 3 * numRecords); // Now try updating w/ the original schema (should succeed) @@ -325,7 +326,7 @@ public void testCopyOnWriteTable(boolean shouldAllowDroppedColumns) throws Excep try { updateBatch(hoodieWriteConfig, client, "008", "007", Option.empty(), initCommitTime, numUpdateRecords, SparkRDDWriteClient::upsert, false, true, - numUpdateRecords, 3 * numRecords, 8); + numUpdateRecords, 3 * numRecords, 8, INSTANT_GENERATOR); assertTrue(shouldAllowDroppedColumns); } catch (HoodieUpsertException e) { assertFalse(shouldAllowDroppedColumns); @@ -360,7 +361,7 @@ private void checkReadRecords(String instantTime, int numExpectedRecords) throws private void checkLatestDeltaCommit(String instantTime) { HoodieTimeline timeline = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, timeline.lastInstant().get().getAction()); - assertEquals(instantTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(instantTime, timeline.lastInstant().get().requestedTime()); } private List generateInsertsWithSchema(String commitTime, int numRecords, String schemaStr) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java index 820ac2754c98..49c43bdab62e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestUpdateSchemaEvolution.java @@ -23,7 +23,6 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordLocation; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.InProcessTimeGenerator; @@ -59,6 +58,7 @@ import java.util.function.Function; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -101,7 +101,7 @@ private WriteStatus prepareFirstRecordCommit(List recordsStrs) throws IO }).collect(); final Path commitFile = new Path(config.getBasePath() + "/.hoodie/" - + HoodieTimeline.makeCommitFileName("100" + "_" + InProcessTimeGenerator.createNewInstantTime())); + + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName("100" + "_" + InProcessTimeGenerator.createNewInstantTime())); HadoopFSUtils.getFs(basePath, HoodieTestUtils.getDefaultStorageConf()).create(commitFile); return statuses.get(0); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java index 72619b8300a6..9c51bceaa0c1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestExternalPathHandling.java @@ -67,6 +67,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.index.HoodieIndex.IndexType.INMEMORY; /** @@ -108,7 +109,8 @@ public void testFlow(FileIdAndNameGenerator fileIdAndNameGenerator, List String filePath1 = getPath(partitionPath1, fileName1); WriteStatus writeStatus1 = createWriteStatus(instantTime1, partitionPath1, filePath1, fileId1); JavaRDD rdd1 = createRdd(Collections.singletonList(writeStatus1)); - metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime1), Option.empty()); + metaClient.getActiveTimeline().transitionReplaceRequestedToInflight( + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime1), Option.empty()); writeClient.commit(instantTime1, rdd1, Option.empty(), HoodieTimeline.REPLACE_COMMIT_ACTION, Collections.emptyMap()); assertFileGroupCorrectness(instantTime1, partitionPath1, filePath1, fileId1, 1); @@ -123,7 +125,8 @@ public void testFlow(FileIdAndNameGenerator fileIdAndNameGenerator, List JavaRDD rdd2 = createRdd(Collections.singletonList(newWriteStatus)); Map> partitionToReplacedFileIds = new HashMap<>(); partitionToReplacedFileIds.put(partitionPath1, Collections.singletonList(fileId1)); - metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime2), Option.empty()); + metaClient.getActiveTimeline().transitionReplaceRequestedToInflight( + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime2), Option.empty()); writeClient.commit(instantTime2, rdd2, Option.empty(), HoodieTimeline.REPLACE_COMMIT_ACTION, partitionToReplacedFileIds); assertFileGroupCorrectness(instantTime2, partitionPath1, filePath2, fileId2, 1); @@ -137,7 +140,8 @@ public void testFlow(FileIdAndNameGenerator fileIdAndNameGenerator, List String filePath3 = getPath(partitionPath2, fileName3); WriteStatus writeStatus3 = createWriteStatus(instantTime3, partitionPath2, filePath3, fileId3); JavaRDD rdd3 = createRdd(Collections.singletonList(writeStatus3)); - metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime3), Option.empty()); + metaClient.getActiveTimeline().transitionReplaceRequestedToInflight( + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime3), Option.empty()); writeClient.commit(instantTime3, rdd3, Option.empty(), HoodieTimeline.REPLACE_COMMIT_ACTION, Collections.emptyMap()); assertFileGroupCorrectness(instantTime3, partitionPath2, filePath3, fileId3, partitionPath2.isEmpty() ? 2 : 1); @@ -146,10 +150,10 @@ public void testFlow(FileIdAndNameGenerator fileIdAndNameGenerator, List String cleanTime = writeClient.createNewInstantTime(); HoodieCleanerPlan cleanerPlan = cleanerPlan(new HoodieActionInstant(instantTime2, HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieInstant.State.COMPLETED.name()), instantTime3, Collections.singletonMap(partitionPath1, Collections.singletonList(new HoodieCleanFileInfo(filePath1, false)))); - metaClient.getActiveTimeline().saveToCleanRequested(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime), + metaClient.getActiveTimeline().saveToCleanRequested(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime), TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); HoodieInstant inflightClean = metaClient.getActiveTimeline().transitionCleanRequestedToInflight( - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime), Option.empty()); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime), Option.empty()); List cleanStats = Collections.singletonList(createCleanStat(partitionPath1, Arrays.asList(filePath1), instantTime2, instantTime3)); HoodieCleanMetadata cleanMetadata = CleanerUtils.convertCleanMetadata( cleanTime, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 488cb082eea2..745f755318d5 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -166,12 +166,15 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_EXTENSION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_EXTENSION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; import static org.apache.hudi.common.table.timeline.HoodieTimeline.INFLIGHT_EXTENSION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REQUESTED_EXTENSION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getNextCommitTime; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; @@ -581,26 +584,26 @@ public void testMetadataTableArchival() throws Exception { // and the "00000000000000" init deltacommit should be archived. HoodieTableMetaClient metadataMetaClient = createMetaClient(metadataTableBasePath); HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); // Trigger clustering in the data table, archival should not kick in, even though conditions are met. doCluster(testTable, metaClient.createNewInstantTime()); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); getHoodieWriteClient(writeConfig); // Trigger a regular write operation. data set timeline archival should kick in. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); archiveDataTable(writeConfig, createMetaClient(basePath)); assertEquals(instants.get(3), - metaClient.reloadActiveTimeline().getCommitsTimeline().firstInstant().get().getTimestamp()); + metaClient.reloadActiveTimeline().getCommitsTimeline().firstInstant().get().requestedTime()); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); // Trigger a regular write operation. metadata timeline archival should kick in. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(3), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(3), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); } @ParameterizedTest @@ -634,7 +637,7 @@ public void testMetadataArchivalCleanConfig(HoodieTableType tableType) throws Ex // and the "00000000000000" init deltacommit should be archived. HoodieTableMetaClient metadataMetaClient = createMetaClient(metadataTableBasePath); HoodieActiveTimeline metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(0), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); getHoodieWriteClient(writeConfig); // Trigger data table archive, should archive 1st, 2nd. @@ -642,7 +645,7 @@ public void testMetadataArchivalCleanConfig(HoodieTableType tableType) throws Ex // Trigger a regular write operation. metadata timeline archival should kick in and catch up with data table. doWriteOperation(testTable, metaClient.createNewInstantTime(), INSERT); metadataTimeline = metadataMetaClient.reloadActiveTimeline(); - assertEquals(instants.get(2), metadataTimeline.getCommitsTimeline().firstInstant().get().getTimestamp()); + assertEquals(instants.get(2), metadataTimeline.getCommitsTimeline().firstInstant().get().requestedTime()); } @ParameterizedTest @@ -785,7 +788,7 @@ public void testMetadataTableDeletePartition(HoodieTableType tableType) throws E Option completedReplaceInstant = metadataMetaClient.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant(); assertTrue(completedReplaceInstant.isPresent()); - assertEquals("0000003", completedReplaceInstant.get().getTimestamp()); + assertEquals("0000003", completedReplaceInstant.get().requestedTime()); final Map metadataEnabledPartitionTypes = new HashMap<>(); metadataWriter.getEnabledPartitionTypes().forEach(e -> metadataEnabledPartitionTypes.put(e.getPartitionPath(), e)); @@ -1103,7 +1106,7 @@ public void testMetadataRollbackWithCompaction() throws Exception { // collect all commit meta files from metadata table. HoodieTableMetaClient metadataMetaClient = HoodieTestUtils.init(storageConf, HoodieTableMetadata.getMetadataTableBasePath(basePath), tableType, new Properties()); - String completionTimeForCommit3 = metadataMetaClient.getActiveTimeline().filter(instant -> instant.getTimestamp().equals(newCommitTime3)).firstInstant() + String completionTimeForCommit3 = metadataMetaClient.getActiveTimeline().filter(instant -> instant.requestedTime().equals(newCommitTime3)).firstInstant() .map(HoodieInstant::getCompletionTime) .orElseThrow(() -> new IllegalStateException(newCommitTime3 + " should exist on the metadata")); String completionTimeForRollback = metadataMetaClient.getActiveTimeline().filter(instant -> instant.getAction().equals(ROLLBACK_ACTION)).firstInstant() @@ -1112,7 +1115,7 @@ public void testMetadataRollbackWithCompaction() throws Exception { // ensure commit2's delta commit in MDT has completion time > the actual rollback for previous failed commit i.e. commit2. // if rollback wasn't eager, rollback's last completion time will be lower than the commit3'd delta commit completion time. - assertTrue(HoodieTimeline.compareTimestamps(completionTimeForCommit3, GREATER_THAN, completionTimeForRollback)); + assertTrue(compareTimestamps(completionTimeForCommit3, GREATER_THAN, completionTimeForRollback)); } } @@ -1172,10 +1175,10 @@ private void revertTableToInflightState(HoodieWriteConfig writeConfig) throws IO assertEquals(1, timeline.getCommitsTimeline().filterCompletedInstants().countInstants()); assertEquals(3, mdtTimeline.countInstants()); assertEquals(3, mdtTimeline.getCommitsTimeline().filterCompletedInstants().countInstants()); - String mdtInitCommit2 = mdtTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().get(1).getTimestamp(); + String mdtInitCommit2 = mdtTimeline.getCommitsTimeline().filterCompletedInstants().getInstants().get(1).requestedTime(); Pair lastCommitMetadataWithValidData = mdtTimeline.getLastCommitMetadataWithValidData().get(); - String commit = lastCommitMetadataWithValidData.getLeft().getTimestamp(); + String commit = lastCommitMetadataWithValidData.getLeft().requestedTime(); assertTrue(timeline.getCommitsTimeline().containsInstant(commit)); assertTrue(mdtTimeline.getCommitsTimeline().containsInstant(commit)); @@ -1206,9 +1209,9 @@ private void revertTableToInflightState(HoodieWriteConfig writeConfig) throws IO metaClient, RECORD_INDEX); timeline = metaClient.getActiveTimeline().reload(); mdtTimeline = mdtMetaClient.getActiveTimeline().reload(); - assertEquals(commit, timeline.lastInstant().get().getTimestamp()); + assertEquals(commit, timeline.lastInstant().get().requestedTime()); assertTrue(timeline.lastInstant().get().isInflight()); - assertEquals(mdtInitCommit2, mdtTimeline.lastInstant().get().getTimestamp()); + assertEquals(mdtInitCommit2, mdtTimeline.lastInstant().get().requestedTime()); assertTrue(mdtTimeline.lastInstant().get().isInflight()); } @@ -1588,7 +1591,7 @@ public void testManualRollbacks() throws Exception { List allInstants = metaClient.reloadActiveTimeline().getCommitsTimeline().getReverseOrderedInstants().collect(Collectors.toList()); for (HoodieInstant instantToRollback : allInstants) { try { - testTable.doRollback(instantToRollback.getTimestamp(), metaClient.createNewInstantTime()); + testTable.doRollback(instantToRollback.requestedTime(), metaClient.createNewInstantTime()); validateMetadata(testTable); } catch (HoodieMetadataException e) { // This is expected since we are rolling back commits that are older than the latest compaction on the MDT @@ -1933,9 +1936,9 @@ public void testColStatsPrefixLookup() throws IOException { // populate commit -> partition -> file info to assist in validation and prefix search metaClient.getActiveTimeline().getInstants().forEach(entry -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); - String commitTime = entry.getTimestamp(); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(entry, metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); + String commitTime = entry.requestedTime(); if (!commitToPartitionsToFiles.containsKey(commitTime)) { commitToPartitionsToFiles.put(commitTime, new HashMap<>()); } @@ -2059,7 +2062,7 @@ public void testEagerRollbackinMDT() throws IOException { .collect(Collectors.toList()); List rollbackFiles = metaFiles.stream() .filter(pathInfo -> - pathInfo.getPath().getName().equals(rollbackInstant.getFileName())) + pathInfo.getPath().getName().equals(INSTANT_FILE_NAME_GENERATOR.getFileName(rollbackInstant))) .collect(Collectors.toList()); // ensure commit3's delta commit in MDT has last mod time > the actual rollback for previous failed commit i.e. commit2. @@ -2222,9 +2225,9 @@ public void testMetadataMultiWriter() throws Exception { // Ensure all commits were synced to the Metadata Table HoodieTableMetaClient metadataMetaClient = createMetaClient(metadataTableBasePath); assertEquals(metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().countInstants(), 5); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000002"))); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000003"))); - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "0000004"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000002"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000003"))); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000004"))); // Compaction may occur if the commits completed in order assertTrue(metadataMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants() <= 1); @@ -2604,7 +2607,11 @@ public void testUpgradeDowngrade() throws IOException { // Perform a commit. This should bootstrap the metadata table with latest version. List records; List writeStatuses; - HoodieWriteConfig writeConfig = getWriteConfig(true, true); + // Set Writer Version 6. + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + tableConfig.setTableVersion(HoodieTableVersion.SIX); + initMetaClient(COPY_ON_WRITE, tableConfig.getProps()); + HoodieWriteConfig writeConfig = getWriteConfigBuilder(true, true, false).withWriteTableVersion(6).build(); String commitTimestamp; try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { commitTimestamp = client.createNewInstantTime(); @@ -2674,6 +2681,7 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3"); properties.setProperty(LockConfiguration.LOCK_ACQUIRE_WAIT_TIMEOUT_MS_PROP_KEY, "3000"); HoodieWriteConfig writeConfig = getWriteConfigBuilder(false, true, false) + .withWriteTableVersion(6) .withCleanConfig(HoodieCleanConfig.newBuilder() .withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.LAZY).withAutoClean(false).build()) .withWriteConcurrencyMode(WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL) @@ -2681,7 +2689,10 @@ public void testRollbackDuringUpgradeForDoubleLocking() throws IOException { .withProperties(properties) .build(); String commitTimestamp; - + // Set Writer Version 6. + HoodieTableConfig tableConfig = metaClient.getTableConfig(); + tableConfig.setTableVersion(HoodieTableVersion.SIX); + initMetaClient(COPY_ON_WRITE, tableConfig.getProps()); try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, writeConfig)) { commitTimestamp = client.createNewInstantTime(); records = dataGen.generateInserts(commitTimestamp, 5); @@ -2768,8 +2779,7 @@ public void testRollbackOfPartiallyFailedCommitWithNewPartitions() throws Except // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = - metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get() - .getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile(new StoragePath( basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName))); } @@ -2994,8 +3004,7 @@ public void testErrorCases() throws Exception { // There is no way to simulate failed commit on the main dataset, hence we simply delete the completed // instant so that only the inflight is left over. String commitInstantFileName = - metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get() - .getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.getActiveTimeline().getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile(new StoragePath( basePath + StoragePath.SEPARATOR + METAFOLDER_NAME, commitInstantFileName))); } @@ -3239,8 +3248,8 @@ public void testRepeatedActionWithSameInstantTime() throws Exception { // To simulate failed clean on the main dataset, we will delete the completed clean instant String cleanInstantFileName = - metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() - .getReverseOrderedInstants().findFirst().get().getFileName(); + INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.reloadActiveTimeline().getCleanerTimeline().filterCompletedInstants() + .getReverseOrderedInstants().findFirst().get()); assertTrue(storage.deleteFile(new StoragePath( basePath + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME, cleanInstantFileName))); assertEquals( @@ -3461,7 +3470,7 @@ public void testDeleteWithRecordIndex() throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); assertTrue(metaClient.getTableConfig().isMetadataPartitionAvailable(RECORD_INDEX), "RI is disabled"); assertEquals(firstBatchOfrecords.size(), - HoodieClientTestUtils.readCommit(writeConfig.getBasePath(), engineContext.getSqlContext(), metaClient.reloadActiveTimeline(), firstCommitTime).count()); + HoodieClientTestUtils.readCommit(writeConfig.getBasePath(), engineContext.getSqlContext(), metaClient.reloadActiveTimeline(), firstCommitTime, true, INSTANT_GENERATOR).count()); // Another batch of records added secondCommitTime = client.createNewInstantTime(); @@ -3470,7 +3479,7 @@ public void testDeleteWithRecordIndex() throws Exception { client.bulkInsert(jsc.parallelize(secondBatchOfrecords, 1), secondCommitTime).collect(); assertEquals(secondBatchOfrecords.size(), - HoodieClientTestUtils.readCommit(writeConfig.getBasePath(), engineContext.getSqlContext(), metaClient.reloadActiveTimeline(), secondCommitTime).count()); + HoodieClientTestUtils.readCommit(writeConfig.getBasePath(), engineContext.getSqlContext(), metaClient.reloadActiveTimeline(), secondCommitTime, true, INSTANT_GENERATOR).count()); allRecords = new ArrayList<>(firstBatchOfrecords); allRecords.addAll(secondBatchOfrecords); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java index 1b135ff7a564..798dc0bdc8c3 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedTableMetadata.java @@ -86,6 +86,7 @@ import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLEAN_ACTION; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; import static org.apache.hudi.metadata.MetadataPartitionType.FILES; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -354,7 +355,7 @@ public void testRepeatedCleanActionsWithMetadataTableEnabled(final HoodieTableTy metaClient.reloadActiveTimeline(); HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan( - metaClient, new HoodieInstant(HoodieInstant.State.REQUESTED, CLEAN_ACTION, cleanInstant)); + metaClient, INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, CLEAN_ACTION, cleanInstant)); testTable.repeatClean(metaClient.createNewInstantTime(), cleanerPlan, cleanMetadata); // Compaction should not happen after the first compaction in this test case @@ -369,7 +370,7 @@ private int getNumCompactions(HoodieTableMetaClient metaClient) { .filter(s -> { try { return s.getAction().equals(HoodieTimeline.COMMIT_ACTION) - && HoodieCommitMetadata.fromBytes( + && metaClient.getCommitMetadataSerDe().deserialize(s, timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class) .getOperationType().equals(COMPACT); } catch (IOException e) { 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 b013287af648..50ec411a3a6e 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 @@ -53,6 +53,8 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; @@ -87,6 +89,7 @@ import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.HoodieWriteMetadata; import org.apache.hudi.table.action.cluster.ClusteringTestUtils; +import org.apache.hudi.table.upgrade.SparkUpgradeDowngradeHelper; import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieSparkWriteableTestTable; @@ -124,11 +127,13 @@ import static org.apache.hudi.common.table.timeline.HoodieInstant.State.REQUESTED; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLUSTERING_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; -import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.testutils.Transformations.randomSelectAsHoodieKeys; import static org.apache.hudi.common.testutils.Transformations.recordsToRecordKeySet; import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; @@ -217,10 +222,11 @@ public void setUpTestTable() { protected Object castInsertFirstBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert, + InstantGenerator instantGenerator) throws Exception { return insertFirstBatch(writeConfig, (SparkRDDWriteClient) client, newCommitTime, initCommitTime, numRecordsInThisCommit, (writeClient, records, commitTime) -> (JavaRDD) writeFn.apply(writeClient, records, commitTime), - isPreppedAPI, assertForCommit, expRecordsInThisCommit, filterForCommitTimeWithAssert); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, filterForCommitTimeWithAssert, instantGenerator); } @Override @@ -229,10 +235,10 @@ protected Object castWriteBatch(BaseHoodieWriteClient client, String newCommitTi Function2, String, Integer> recordGenFunction, Function3 writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { return writeBatch((SparkRDDWriteClient) client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, (writeClient, records, commitTime) -> (JavaRDD) writeFn.apply(writeClient, records, commitTime), - assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, filterForCommitTimeWithAssert); + assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } @Override @@ -240,21 +246,22 @@ protected Object castUpdateBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteC Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3 writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); return writeBatch((SparkRDDWriteClient) client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, (writeClient, records, commitTime) -> (JavaRDD) writeFn.apply(writeClient, records, commitTime), assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, false, filterForCommitTimeWithAssert); + expTotalCommits, false, filterForCommitTimeWithAssert, instantGenerator); } @Override protected Object castDeleteBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + boolean filterForCommitTimeWithAssert, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return deleteBatch(writeConfig, (SparkRDDWriteClient) client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, - isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, timelineFactory, instantGenerator); } /** @@ -263,7 +270,7 @@ protected Object castDeleteBatch(HoodieWriteConfig writeConfig, BaseHoodieWriteC @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testAutoCommitOnInsert(boolean populateMetaFields) throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insert(recordRDD, instantTime), false, populateMetaFields); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insert(recordRDD, instantTime), false, populateMetaFields, INSTANT_GENERATOR); } /** @@ -271,7 +278,7 @@ public void testAutoCommitOnInsert(boolean populateMetaFields) throws Exception */ @Test public void testAutoCommitOnInsertPrepped() throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insertPreppedRecords(recordRDD, instantTime), true, true); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.insertPreppedRecords(recordRDD, instantTime), true, true, INSTANT_GENERATOR); } /** @@ -280,7 +287,7 @@ public void testAutoCommitOnInsertPrepped() throws Exception { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testAutoCommitOnUpsert(boolean populateMetaFields) throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), false, populateMetaFields); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), false, populateMetaFields, INSTANT_GENERATOR); } /** @@ -289,7 +296,7 @@ public void testAutoCommitOnUpsert(boolean populateMetaFields) throws Exception @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testAutoCommitOnUpsertPrepped(boolean populateMetaFields) throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, populateMetaFields); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, populateMetaFields, INSTANT_GENERATOR); } /** @@ -298,7 +305,7 @@ public void testAutoCommitOnUpsertPrepped(boolean populateMetaFields) throws Exc @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testAutoCommitOnBulkInsert(boolean populateMetaFields) throws Exception { - testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime), false, populateMetaFields); + testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime), false, populateMetaFields, INSTANT_GENERATOR); } /** @@ -307,7 +314,7 @@ public void testAutoCommitOnBulkInsert(boolean populateMetaFields) throws Except @Test public void testAutoCommitOnBulkInsertPrepped() throws Exception { testAutoCommit((writeClient, recordRDD, instantTime) -> writeClient.bulkInsertPreppedRecords(recordRDD, instantTime, - Option.empty()), true, true); + Option.empty()), true, true, INSTANT_GENERATOR); } @Test @@ -320,7 +327,7 @@ public void testPreCommitValidatorsOnInsert() throws Exception { writeClient.bulkInsert(recordRDD, instantTime, Option.empty()); String newCommitTime = client.createNewInstantTime(); JavaRDD result = insertFirstBatch(config, client, newCommitTime, - "000", numRecords, writeFn, false, false, numRecords); + "000", numRecords, writeFn, false, false, numRecords, INSTANT_GENERATOR); assertTrue(testTable.commitExists(newCommitTime)); } } @@ -335,7 +342,7 @@ public void testPreCommitValidationFailureOnInsert() throws Exception { Function3, SparkRDDWriteClient, JavaRDD, String> writeFn = (writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime, Option.empty()); JavaRDD result = insertFirstBatch(config, client, newCommitTime, - "000", numRecords, writeFn, false, false, numRecords); + "000", numRecords, writeFn, false, false, numRecords, INSTANT_GENERATOR); fail("Expected validation to fail because we only insert 200 rows. Validation is configured to expect 500 rows"); } catch (HoodieInsertException e) { if (e.getCause() instanceof HoodieValidationException) { @@ -389,7 +396,7 @@ private void insertWithConfig(HoodieWriteConfig config, int numRecords, String i Function3, SparkRDDWriteClient, JavaRDD, String> writeFn = (writeClient, recordRDD, instantTime) -> writeClient.bulkInsert(recordRDD, instantTime, Option.empty()); JavaRDD result = insertFirstBatch(config, client, instant, - "000", numRecords, writeFn, false, false, numRecords); + "000", numRecords, writeFn, false, false, numRecords, INSTANT_GENERATOR); } } @@ -434,7 +441,7 @@ public void testDeduplicationOnUpsert() throws Exception { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testUpserts(boolean populateMetaFields) throws Exception { - testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), populateMetaFields, false); + testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsert(recordRDD, instantTime), populateMetaFields, false, SparkUpgradeDowngradeHelper.getInstance()); } /** @@ -442,7 +449,7 @@ public void testUpserts(boolean populateMetaFields) throws Exception { */ @Test public void testUpsertsPrepped() throws Exception { - testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true); + testUpsertsInternal((writeClient, recordRDD, instantTime) -> writeClient.upsertPreppedRecords(recordRDD, instantTime), true, true, SparkUpgradeDowngradeHelper.getInstance()); } @Override @@ -510,12 +517,10 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { HoodieWriteConfig hoodieWriteConfig = getConfigBuilder(EAGER) .withRollbackUsingMarkers(true) .withArchivalConfig(HoodieArchivalConfig.newBuilder().withArchiveBeyondSavepoint(true).build()) - .withProps(config.getProps()).withTimelineLayoutVersion( - VERSION_0).build(); + .withProps(config.getProps()).build(); HoodieTableMetaClient.newTableBuilder() .fromMetaClient(metaClient) - .setTimelineLayoutVersion(VERSION_0) .setPopulateMetaFields(config.populateMetaFields()) .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); @@ -526,7 +531,7 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { String initCommitTime = "000"; int numRecords = 200; insertFirstBatch(hoodieWriteConfig, client, newCommitTime, initCommitTime, numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, config.populateMetaFields()); + false, true, numRecords, config.populateMetaFields(), INSTANT_GENERATOR); // Write 2 (updates) String prevCommitTime = newCommitTime; @@ -535,7 +540,7 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { String commitTimeBetweenPrevAndNew = "002"; updateBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), initCommitTime, numRecords, SparkRDDWriteClient::upsert, false, true, - numRecords, 200, 2, config.populateMetaFields()); + numRecords, 200, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Delete 1 prevCommitTime = newCommitTime; @@ -544,7 +549,7 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { deleteBatch(hoodieWriteConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecords, false, true, - 0, 150, config.populateMetaFields()); + 0, 150, config.populateMetaFields(), TIMELINE_FACTORY, INSTANT_GENERATOR); HoodieWriteConfig newConfig = getConfigBuilder().withProps(config.getProps()).withTimelineLayoutVersion( TimelineLayoutVersion.CURR_VERSION) @@ -565,7 +570,7 @@ public void testRestoreWithSavepointBeyondArchival() throws Exception { @ParameterizedTest @MethodSource("populateMetaFieldsParams") public void testInsertsWithHoodieConcatHandle(boolean populateMetaFields) throws Exception { - testHoodieConcatHandle(populateMetaFields, false); + testHoodieConcatHandle(populateMetaFields, false, INSTANT_GENERATOR); } /** @@ -573,7 +578,7 @@ public void testInsertsWithHoodieConcatHandle(boolean populateMetaFields) throws */ @Test public void testInsertsPreppedWithHoodieConcatHandle() throws Exception { - testHoodieConcatHandle(true, true); + testHoodieConcatHandle(true, true, INSTANT_GENERATOR); } /** @@ -581,7 +586,7 @@ public void testInsertsPreppedWithHoodieConcatHandle() throws Exception { */ @Test public void testInsertsWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Exception { - testHoodieConcatHandleOnDupInserts(false); + testHoodieConcatHandleOnDupInserts(false, INSTANT_GENERATOR); } /** @@ -589,7 +594,7 @@ public void testInsertsWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Ex */ @Test public void testInsertsPreppedWithHoodieConcatHandleOnDuplicateIncomingKeys() throws Exception { - testHoodieConcatHandleOnDupInserts(true); + testHoodieConcatHandleOnDupInserts(true, INSTANT_GENERATOR); } @Test @@ -625,10 +630,10 @@ public void testPendingRestore() throws IOException { .getInstants().get(0); completeRestoreFile = new StoragePath( config.getBasePath() + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME - + StoragePath.SEPARATOR + restoreCompleted.getFileName()); + + StoragePath.SEPARATOR + INSTANT_FILE_NAME_GENERATOR.getFileName(restoreCompleted)); backupCompletedRestoreFile = new StoragePath( config.getBasePath() + StoragePath.SEPARATOR + HoodieTableMetaClient.METAFOLDER_NAME - + StoragePath.SEPARATOR + restoreCompleted.getFileName() + ".backup"); + + StoragePath.SEPARATOR + INSTANT_FILE_NAME_GENERATOR.getFileName(restoreCompleted) + ".backup"); metaClient.getStorage().rename(completeRestoreFile, backupCompletedRestoreFile); } @@ -673,7 +678,7 @@ public void testDeletes() throws Exception { */ @Test public void testDeletesForInsertsInSameBatch() throws Exception { - super.testDeletesForInsertsInSameBatch(); + super.testDeletesForInsertsInSameBatch(INSTANT_GENERATOR); } private Pair, List> insertBatchRecords(SparkRDDWriteClient client, String commitTime, @@ -1071,28 +1076,28 @@ public void testPendingClusteringRollback() throws Exception { allRecords.addAll(dataGen.generateInserts(commitTime, 200)); assertThrows(HoodieUpsertException.class, () -> writeAndVerifyBatch(client, allRecords, commitTime, populateMetaFields)); // verify pending clustering can be rolled back (even though there is a completed commit greater than pending clustering) - client.rollback(pendingClusteringInstant.getTimestamp()); + client.rollback(pendingClusteringInstant.requestedTime()); metaClient.reloadActiveTimeline(); // verify there are no pending clustering instants assertEquals(0, ClusteringUtils.getAllPendingClusteringPlans(metaClient).count()); // delete rollback.completed instant to mimic failed rollback of clustering. and then trigger rollback of clustering again. same rollback instant should be used. HoodieInstant rollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get(); - FileCreateUtils.deleteRollbackCommit(metaClient.getBasePath().toString(), rollbackInstant.getTimestamp()); + FileCreateUtils.deleteRollbackCommit(metaClient.getBasePath().toString(), rollbackInstant.requestedTime()); metaClient.reloadActiveTimeline(); // create replace commit requested meta file so that rollback will not throw FileNotFoundException // create file slice with instantTime 001 and build clustering plan including this created 001 file slice. - HoodieClusteringPlan clusteringPlan = ClusteringTestUtils.createClusteringPlan(metaClient, pendingClusteringInstant.getTimestamp(), "1"); + HoodieClusteringPlan clusteringPlan = ClusteringTestUtils.createClusteringPlan(metaClient, pendingClusteringInstant.requestedTime(), "1"); // create requested replace commit HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); - FileCreateUtils.createRequestedClusterCommit(metaClient.getBasePath().toString(), pendingClusteringInstant.getTimestamp(), requestedReplaceMetadata); + FileCreateUtils.createRequestedClusterCommit(metaClient.getBasePath().toString(), pendingClusteringInstant.requestedTime(), requestedReplaceMetadata); // trigger clustering again. no new rollback instants should be generated. try { - client.cluster(pendingClusteringInstant.getTimestamp(), false); + client.cluster(pendingClusteringInstant.requestedTime(), false); // new replace commit metadata generated is fake one. so, clustering will fail. but the intention of test is ot check for duplicate rollback instants. } catch (Exception e) { //ignore. @@ -1101,7 +1106,7 @@ public void testPendingClusteringRollback() throws Exception { metaClient.reloadActiveTimeline(); // verify that there is no new rollback instant generated HoodieInstant newRollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get(); - assertEquals(rollbackInstant.getTimestamp(), newRollbackInstant.getTimestamp()); + assertEquals(rollbackInstant.requestedTime(), newRollbackInstant.requestedTime()); } @ParameterizedTest @@ -1541,13 +1546,13 @@ public void testClusteringCommitInPresenceOfInflightCommit() throws Exception { HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {partitionStr}); writeBatch(client, firstCommit, "000", Option.of(Arrays.asList("000")), "000", numRecords, dataGenerator::generateInserts, SparkRDDWriteClient::insert, true, numRecords, numRecords, - 1, true); + 1, true, INSTANT_GENERATOR); // Do an upsert operation without autocommit. String inflightCommit = client.createNewInstantTime(); writeBatch(client, inflightCommit, firstCommit, Option.of(Arrays.asList("000")), "000", 100, dataGenerator::generateUniqueUpdates, SparkRDDWriteClient::upsert, false, 0, 200, - 2, false); + 2, false, INSTANT_GENERATOR); // Schedule and execute a clustering plan on the same partition. During conflict resolution the commit should fail. HoodieWriteConfig clusteringWriteConfig = getConfigBuilder().withAutoCommit(false) @@ -1574,7 +1579,7 @@ public void testClusteringCommitInPresenceOfInflightCommit() throws Exception { List instants = metaClient.reloadActiveTimeline().getInstants(); assertEquals(3, instants.size()); assertEquals(HoodieActiveTimeline.ROLLBACK_ACTION, instants.get(2).getAction()); - assertEquals(new HoodieInstant(true, HoodieActiveTimeline.COMMIT_ACTION, inflightCommit), instants.get(1)); + assertEquals(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieActiveTimeline.COMMIT_ACTION, inflightCommit), instants.get(1)); } @Test @@ -1597,13 +1602,13 @@ public void testIngestionCommitInPresenceOfCompletedClusteringCommit() throws Ex HoodieTestDataGenerator dataGenerator = new HoodieTestDataGenerator(new String[] {partitionStr}); writeBatch(client, firstCommit, "000", Option.of(Arrays.asList("000")), "000", numRecords, dataGenerator::generateInserts, SparkRDDWriteClient::insert, true, numRecords, numRecords, - 1, true); + 1, true, INSTANT_GENERATOR); // Create and temporarily block a lower timestamp for ingestion. String inflightCommit = client.createNewInstantTime(); JavaRDD ingestionResult = writeBatch(client, inflightCommit, firstCommit, Option.of(Arrays.asList("000")), "000", 100, dataGenerator::generateUniqueUpdates, SparkRDDWriteClient::upsert, false, 0, 200, - 2, false); + 2, false, INSTANT_GENERATOR); // Schedule and execute a clustering plan on the same partition. During conflict resolution the commit should fail. // Since it is harder to test corner cases where the ingestion writer is at dedupe step right before the inflight file creation @@ -1635,7 +1640,7 @@ protected HoodieInstant createRequestedClusterInstant(HoodieTableMetaClient meta HoodieClusteringPlan clusteringPlan = ClusteringUtils.createClusteringPlan(EXECUTION_STRATEGY_CLASS_NAME.defaultValue(), STRATEGY_PARAMS, fileSlices, Collections.emptyMap()); - HoodieInstant clusteringInstant = new HoodieInstant(REQUESTED, CLUSTERING_ACTION, clusterTime); + HoodieInstant clusteringInstant = INSTANT_GENERATOR.createNewInstant(REQUESTED, CLUSTERING_ACTION, clusterTime); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); metaClient.getActiveTimeline().saveToPendingClusterCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); 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 6354ab175de8..d7e1589aa854 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 @@ -57,7 +57,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.assertDataInMORTable; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -83,20 +85,20 @@ public void testReadingMORTableWithoutBaseFile() throws Exception { // Insert String commitTime = client.createNewInstantTime(); insertBatch(config, client, commitTime, "000", 100, SparkRDDWriteClient::insert, - false, false, 100, 100, 1, Option.empty()); + false, false, 100, 100, 1, Option.empty(), INSTANT_GENERATOR); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Delete 5 records String prevCommitTime = commitTime; commitTime = client.createNewInstantTime(); deleteBatch(config, client, commitTime, prevCommitTime, "000", 25, false, false, - 0, 100); + 0, 100, TIMELINE_FACTORY, INSTANT_GENERATOR); // Verify all the records. metaClient.reloadActiveTimeline(); @@ -116,14 +118,14 @@ public void testCompactionOnMORTable() throws Exception { // Insert String commitTime = client.createNewInstantTime(); insertBatch(config, client, commitTime, "000", 100, SparkRDDWriteClient::insert, - false, false, 100, 100, 1, Option.empty()); + false, false, 100, 100, 1, Option.empty(), INSTANT_GENERATOR); // Update String commitTimeBetweenPrevAndNew = commitTime; commitTime = client.createNewInstantTime(); updateBatch(config, client, commitTime, commitTimeBetweenPrevAndNew, Option.of(Arrays.asList(commitTimeBetweenPrevAndNew)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 5, 100, 2, config.populateMetaFields()); + false, false, 5, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule and execute compaction. Option timeStamp = client.scheduleCompaction(Option.empty()); @@ -150,7 +152,7 @@ public void testLogCompactionOnMORTable() throws Exception { String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", expectedTotalRecs, SparkRDDWriteClient::insert, false, false, expectedTotalRecs, expectedTotalRecs, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); String prevCommitTime = newCommitTime; for (int i = 0; i < 5; i++) { @@ -159,7 +161,7 @@ public void testLogCompactionOnMORTable() throws Exception { expectedTotalRecs += 50; updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, expectedTotalRecs, i + 2, config.populateMetaFields()); + false, false, 50, expectedTotalRecs, i + 2, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; } @@ -175,7 +177,7 @@ public void testLogCompactionOnMORTable() throws Exception { expectedTotalRecs += 50; updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, expectedTotalRecs, i + 8, config.populateMetaFields()); + false, false, 50, expectedTotalRecs, i + 8, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; } String lastCommitBeforeLogCompaction = prevCommitTime; @@ -208,20 +210,20 @@ public void testLogCompactionOnMORTableWithoutBaseFile() throws Exception { String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 100, 100, - 1, Option.of(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)); + 1, Option.of(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH), INSTANT_GENERATOR); // Upsert 5 records String prevCommitTime = newCommitTime; newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; // Delete 3 records newCommitTime = client.createNewInstantTime(); deleteBatch(config, client, newCommitTime, prevCommitTime, "000", 30, false, false, - 0, 70); + 0, 70, TIMELINE_FACTORY, INSTANT_GENERATOR); String lastCommitBeforeLogCompaction = newCommitTime; // Schedule and execute compaction. @@ -250,14 +252,14 @@ public void testSchedulingLogCompactionAfterSchedulingCompaction() throws Except String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 100, 100, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); String prevCommitTime = newCommitTime; // Upsert newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule compaction Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); @@ -294,14 +296,14 @@ public void testSchedulingCompactionAfterSchedulingLogCompaction() throws Except String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 10, 100, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); String prevCommitTime = newCommitTime; // Upsert newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 2, config.populateMetaFields()); + false, false, 50, 10, 2, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule log compaction Option logCompactionTimeStamp = client.scheduleLogCompaction(Option.empty()); @@ -327,7 +329,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 100, 100, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); // Schedule and execute compaction. Here, second file slice gets added. Option compactionTimeStamp = client.scheduleCompaction(Option.empty()); @@ -339,7 +341,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; // Schedule compaction. Third file slice gets added, compaction is not complete so base file is not created yet. @@ -352,7 +354,7 @@ public void testCleanFunctionalityWhenCompactionRequestedInstantIsPresent() thro newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 100, 2, config.populateMetaFields()); + false, false, 50, 100, 2, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; if (i == 2) { // Since retain commits is 4 exactly after 6th completed commit there will be some files to be cleaned, @@ -383,14 +385,14 @@ public void testRollbackOnLogCompaction() throws Exception { String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 100, 100, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); String prevCommitTime = newCommitTime; // Upsert newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, - false, false, 10, 100, 4, config.populateMetaFields()); + false, false, 10, 100, 4, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; // Schedule and execute log-compaction but do not commit. @@ -399,7 +401,7 @@ public void testRollbackOnLogCompaction() throws Exception { lcClient.logCompact(logCompactionTimeStamp.get()); // Rollback the log compaction commit. - HoodieInstant instant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionTimeStamp.get()); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, logCompactionTimeStamp.get()); getHoodieTable(metaClient, config).rollbackInflightLogCompaction(instant); // Validate timeline. @@ -409,14 +411,14 @@ public void testRollbackOnLogCompaction() throws Exception { assertEquals(HoodieTimeline.ROLLBACK_ACTION, rollbackInstant.getAction()); // Validate block instant times. - validateBlockInstantsBeforeAndAfterRollback(config, prevCommitTime, rollbackInstant.getTimestamp()); - prevCommitTime = rollbackInstant.getTimestamp(); + validateBlockInstantsBeforeAndAfterRollback(config, prevCommitTime, rollbackInstant.requestedTime()); + prevCommitTime = rollbackInstant.requestedTime(); // Do one more upsert newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 10, SparkRDDWriteClient::upsert, - false, false, 10, 100, 4, config.populateMetaFields()); + false, false, 10, 100, 4, config.populateMetaFields(), INSTANT_GENERATOR); prevCommitTime = newCommitTime; // Complete log-compaction now. @@ -493,7 +495,7 @@ public void testArchivalOnLogCompaction() throws Exception { String newCommitTime = client.createNewInstantTime(); insertBatch(config, client, newCommitTime, "000", 100, SparkRDDWriteClient::insert, false, false, 10, 100, - 1, Option.empty()); + 1, Option.empty(), INSTANT_GENERATOR); String prevCommitTime = newCommitTime; List logCompactionInstantTimes = new ArrayList<>(); @@ -510,7 +512,7 @@ public void testArchivalOnLogCompaction() throws Exception { newCommitTime = client.createNewInstantTime(); updateBatch(config, client, newCommitTime, prevCommitTime, Option.of(Arrays.asList(prevCommitTime)), "000", 50, SparkRDDWriteClient::upsert, - false, false, 50, 10, 0, config.populateMetaFields()); + false, false, 50, 10, 0, config.populateMetaFields(), INSTANT_GENERATOR); // Schedule log compaction. Option logCompactionTimeStamp = lcWriteClient.scheduleLogCompaction(Option.empty()); if (logCompactionTimeStamp.isPresent()) { @@ -521,7 +523,7 @@ public void testArchivalOnLogCompaction() throws Exception { } boolean logCompactionInstantArchived = false; Map> instantsMap = metaClient.getArchivedTimeline().getInstantsAsStream() - .collect(Collectors.groupingBy(HoodieInstant::getTimestamp)); + .collect(Collectors.groupingBy(HoodieInstant::requestedTime)); for (String logCompactionTimeStamp : logCompactionInstantTimes) { List instants = instantsMap.get(logCompactionTimeStamp); if (instants == null) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java index ce6330169532..e1a6a5c62f98 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieIndex.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -82,6 +81,8 @@ import scala.Tuple2; import static org.apache.hudi.avro.HoodieAvroUtils.addMetadataFields; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.deleteMetadataPartition; import static org.apache.hudi.metadata.HoodieTableMetadataUtil.metadataPartitionExists; import static org.apache.hudi.metadata.MetadataPartitionType.COLUMN_STATS; @@ -530,20 +531,20 @@ public void testCheckIfValidCommit() throws Exception { setUp(IndexType.BLOOM, true, false); // When timeline is empty, all commits are invalid - HoodieTimeline timeline = new HoodieDefaultTimeline(Collections.EMPTY_LIST.stream(), metaClient.getActiveTimeline()::getInstantDetails); + HoodieTimeline timeline = TIMELINE_FACTORY.createDefaultTimeline(Collections.EMPTY_LIST.stream(), metaClient.getActiveTimeline()::getInstantDetails); assertTrue(timeline.empty()); assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, "001")); assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, writeClient.createNewInstantTime())); assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // Valid when timeline contains the timestamp or the timestamp is before timeline start - final HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "010"); + final HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "010"); String instantTimestamp = writeClient.createNewInstantTime(); - final HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, writeClient.createNewInstantTime()); - timeline = new HoodieDefaultTimeline(Stream.of(instant1, instant2), metaClient.getActiveTimeline()::getInstantDetails); + final HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, writeClient.createNewInstantTime()); + timeline = TIMELINE_FACTORY.createDefaultTimeline(Stream.of(instant1, instant2), metaClient.getActiveTimeline()::getInstantDetails); assertFalse(timeline.empty()); - assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instant1.getTimestamp())); - assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instant2.getTimestamp())); + assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instant1.requestedTime())); + assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instant2.requestedTime())); // no instant on timeline assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, instantTimestamp)); // future timestamp @@ -555,16 +556,16 @@ public void testCheckIfValidCommit() throws Exception { // Check for older timestamp which have sec granularity and an extension of DEFAULT_MILLIS_EXT may have been added via Timeline operations instantTimestamp = writeClient.createNewInstantTime(); String instantTimestampSec = instantTimestamp.substring(0, instantTimestamp.length() - HoodieInstantTimeGenerator.DEFAULT_MILLIS_EXT.length()); - final HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTimestampSec); - timeline = new HoodieDefaultTimeline(Stream.of(instant1, instant3), metaClient.getActiveTimeline()::getInstantDetails); + final HoodieInstant instant3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTimestampSec); + timeline = TIMELINE_FACTORY.createDefaultTimeline(Stream.of(instant1, instant3), metaClient.getActiveTimeline()::getInstantDetails); assertFalse(timeline.empty()); assertFalse(HoodieIndexUtils.checkIfValidCommit(timeline, instantTimestamp)); assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instantTimestampSec)); // With a sec format instant time lesser than first entry in the active timeline, checkifContainsOrBefore() should return true instantTimestamp = writeClient.createNewInstantTime(); - final HoodieInstant instant4 = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTimestamp); - timeline = new HoodieDefaultTimeline(Stream.of(instant4), metaClient.getActiveTimeline()::getInstantDetails); + final HoodieInstant instant4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTimestamp); + timeline = TIMELINE_FACTORY.createDefaultTimeline(Stream.of(instant4), metaClient.getActiveTimeline()::getInstantDetails); instantTimestampSec = instantTimestamp.substring(0, instantTimestamp.length() - HoodieInstantTimeGenerator.DEFAULT_MILLIS_EXT.length()); assertFalse(timeline.empty()); assertTrue(HoodieIndexUtils.checkIfValidCommit(timeline, instantTimestamp)); @@ -586,18 +587,19 @@ public void testCheckIfValidCommit() throws Exception { Thread.sleep(2000); // sleep required so that new timestamp differs in the seconds rather than msec String newTimestamp = writeClient.createNewInstantTime(); String newTimestampSec = newTimestamp.substring(0, newTimestamp.length() - HoodieInstantTimeGenerator.DEFAULT_MILLIS_EXT.length()); - final HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, newTimestamp); - timeline = new HoodieDefaultTimeline(Stream.of(instant5), metaClient.getActiveTimeline()::getInstantDetails); + final HoodieInstant instant5 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, newTimestamp); + timeline = TIMELINE_FACTORY.createDefaultTimeline(Stream.of(instant5), metaClient.getActiveTimeline()::getInstantDetails); assertFalse(timeline.empty()); assertFalse(timeline.containsInstant(checkInstantTimestamp)); assertFalse(timeline.containsInstant(checkInstantTimestampSec)); - final HoodieInstant instant6 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, newTimestampSec + HoodieInstantTimeGenerator.DEFAULT_MILLIS_EXT); - timeline = new HoodieDefaultTimeline(Stream.of(instant6), metaClient.getActiveTimeline()::getInstantDetails); + final HoodieInstant instant6 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, + HoodieTimeline.DELTA_COMMIT_ACTION, newTimestampSec + HoodieInstantTimeGenerator.DEFAULT_MILLIS_EXT); + timeline = TIMELINE_FACTORY.createDefaultTimeline(Stream.of(instant6), metaClient.getActiveTimeline()::getInstantDetails); assertFalse(timeline.empty()); assertFalse(timeline.containsInstant(newTimestamp)); assertFalse(timeline.containsInstant(checkInstantTimestamp)); - assertTrue(timeline.containsInstant(instant6.getTimestamp())); + assertTrue(timeline.containsInstant(instant6.requestedTime())); } @Test diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index ee3f0ce04642..1c7a929ee911 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -19,6 +19,7 @@ package org.apache.hudi.client.functional; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.model.HoodieCommitMetadata; @@ -290,7 +291,7 @@ protected void doPreBootstrapRestore(HoodieTestTable testTable, String restoreTi protected void archiveDataTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws IOException { HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(writeConfig, table); archiver.archiveIfRequired(context); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java index 2f68f8f85f6b..a62e2f068391 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java @@ -72,6 +72,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; import static org.apache.hudi.common.table.view.FileSystemViewStorageConfig.REMOTE_PORT_NUM; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -183,7 +184,7 @@ private void runAssertionsForBasePath(boolean useExistingTimelineServer, String // For all the file groups compacted by the compaction commit, the file system view // should return the latest file slices which is written by the latest commit - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieCommitMetadata commitMetadata = COMMIT_METADATA_SER_DE.deserialize(compactionCommit, timeline.getInstantDetails(compactionCommit).get(), HoodieCommitMetadata.class); List> partitionFileIdPairList = commitMetadata.getPartitionToWriteStats().entrySet().stream().flatMap( @@ -209,7 +210,7 @@ private void runAssertionsForBasePath(boolean useExistingTimelineServer, String new RemoteHoodieTableFileSystemView("localhost", timelineServerPort, newMetaClient); List callableList = lookupList.stream() - .map(pair -> new TestViewLookUpCallable(view, pair, compactionCommit.getTimestamp(), basePathStr)) + .map(pair -> new TestViewLookUpCallable(view, pair, compactionCommit.requestedTime(), basePathStr)) .collect(Collectors.toList()); List> resultList = new ArrayList<>(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java index 6a31639f13b5..2b5241ecd18f 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestSavepointRestoreCopyOnWrite.java @@ -35,6 +35,7 @@ import java.util.Objects; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -61,7 +62,7 @@ void testBasicRollback() throws Exception { String newCommitTime = client.createNewInstantTime(); // Write 4 inserts with the 2nd commit been rolled back insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, numRecords * i, 1, Option.empty()); + false, true, numRecords, numRecords * i, 1, Option.empty(), INSTANT_GENERATOR); prevInstant = newCommitTime; if (i == 2) { // trigger savepoint @@ -96,7 +97,7 @@ void testCleaningPendingInstants() throws Exception { String newCommitTime = client.createNewInstantTime(); // Write 4 inserts with the 2nd commit been rolled back insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, numRecords * i, 1, Option.empty()); + false, true, numRecords, numRecords * i, 1, Option.empty(), INSTANT_GENERATOR); prevInstant = newCommitTime; if (i == 2) { // trigger savepoint @@ -139,7 +140,7 @@ void testCleaningRollbackInstants(boolean commitRollback) throws Exception { String newCommitTime = client.createNewInstantTime(); // Write 4 inserts with the 2nd commit been rolled back insertBatch(hoodieWriteConfig, client, newCommitTime, prevInstant, numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, numRecords * i, 1, Option.empty()); + false, true, numRecords, numRecords * i, 1, Option.empty(), INSTANT_GENERATOR); prevInstant = newCommitTime; if (i == 2) { // trigger savepoint @@ -159,12 +160,12 @@ void testCleaningRollbackInstants(boolean commitRollback) throws Exception { HoodieSparkTable.create(client.getConfig(), context) .scheduleRollback(context, client.createNewInstantTime(), pendingInstant, false, true, false); } - Option rollbackInstant = metaClient.reloadActiveTimeline().getRollbackTimeline().lastInstant().map(HoodieInstant::getTimestamp); + Option rollbackInstant = metaClient.reloadActiveTimeline().getRollbackTimeline().lastInstant().map(HoodieInstant::requestedTime); assertTrue(rollbackInstant.isPresent(), "The latest instant should be a rollback"); // write another batch insertBatch(hoodieWriteConfig, client, client.createNewInstantTime(), rollbackInstant.get(), numRecords, SparkRDDWriteClient::insert, - false, true, numRecords, numRecords * 3, 1, Option.empty()); + false, true, numRecords, numRecords * 3, 1, Option.empty(), INSTANT_GENERATOR); // restore client.restoreToSavepoint(Objects.requireNonNull(savepointCommit, "restore commit should not be null")); assertRowNumberEqualsTo(20); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java index 43f0f4c9e5a8..7cdac267802a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/index/hbase/TestSparkHoodieHBaseIndex.java @@ -560,7 +560,7 @@ public void testHbaseTagLocationForArchivedCommits() throws Exception { // make first commit with 20 records JavaRDD writeRecords1 = generateAndCommitRecords(writeClient, 20); metaClient = HoodieTableMetaClient.reload(metaClient); - String commit1 = metaClient.getActiveTimeline().firstInstant().get().getTimestamp(); + String commit1 = metaClient.getActiveTimeline().firstInstant().get().requestedTime(); // Make 6 additional commits, so that first commit is archived for (int nCommit = 0; nCommit < 6; nCommit++) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java index ad612ee5c9b9..61d7e1e27563 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergeHandle.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -56,6 +55,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotEquals; @@ -121,12 +122,12 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap // verify that there is a commit metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + HoodieTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting a single commit."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 001"); + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be 001"); assertEquals(records.size(), - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, INSTANT_GENERATOR).count(), "Must contain 44 records"); /** @@ -147,9 +148,9 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap // verify that there are 2 commits metaClient = HoodieTableMetaClient.reload(metaClient); - timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); assertEquals(2, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting two commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 002"); + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be 002"); Dataset dataSet = getRecords(); assertEquals(45, dataSet.count(), "Must contain 45 records"); @@ -167,9 +168,9 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap // verify that there are now 3 commits metaClient = HoodieTableMetaClient.reload(metaClient); - timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); assertEquals(3, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting three commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), "Latest commit should be 003"); + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be 003"); dataSet = getRecords(); assertEquals(47, dataSet.count(), "Must contain 47 records"); @@ -197,9 +198,9 @@ public void testUpsertsForMultipleRecordsInSameFile(ExternalSpillableMap.DiskMap assertNoWriteErrors(statuses); // verify there are now 4 commits - timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); assertEquals(4, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting four commits."); - assertEquals(timeline.lastInstant().get().getTimestamp(), newCommitTime, "Latest commit should be 004"); + assertEquals(timeline.lastInstant().get().requestedTime(), newCommitTime, "Latest commit should be 004"); // Check the entire dataset has 47 records still dataSet = getRecords(); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java index 39e624f908c7..6b511d5e81b7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieMergedReadHandle.java @@ -198,7 +198,7 @@ private void doMergedReadAndValidate(HoodieTableMetaClient metaClient, HoodieWri .map(baseFile -> Pair.of(partition, baseFile.getFileId())) .collect(Collectors.toList()); assertEquals(1, partitionPathAndFileIDPairs.size()); - String latestCommitTime = table.getActiveTimeline().lastInstant().get().getTimestamp(); + String latestCommitTime = table.getActiveTimeline().lastInstant().get().requestedTime(); HoodieMergedReadHandle mergedReadHandle = new HoodieMergedReadHandle<>(writeConfig, Option.of(latestCommitTime), table, partitionPathAndFileIDPairs.get(0)); List mergedRecords = mergedReadHandle.getMergedRecords(); assertEquals(totalRecords, mergedRecords.size()); 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 435b5543f1c9..28b8c16b2fd8 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 @@ -21,8 +21,8 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.BaseHoodieWriteClient; -import org.apache.hudi.client.timeline.HoodieTimelineArchiver; -import org.apache.hudi.client.timeline.LSMTimelineWriter; +import org.apache.hudi.client.timeline.versioning.v2.LSMTimelineWriter; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.model.HoodieCleaningPolicy; @@ -42,6 +42,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.LSMTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; @@ -107,8 +108,13 @@ import java.util.stream.Stream; import static org.apache.hudi.HoodieTestCommitGenerator.getBaseFilename; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertCommitMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.testutils.HoodieTestUtils.createCompactionCommitInMetadataTable; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.config.HoodieArchivalConfig.ARCHIVE_BEYOND_SAVEPOINT; @@ -260,7 +266,7 @@ public void testArchiveEmptyTable() throws Exception { .withParallelism(2, 2).forTable("test-trip-table").build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); int result = archiver.archiveIfRequired(context); assertEquals(0, result); } @@ -383,7 +389,7 @@ private HoodieInstant triggerCommit( String file1P0C0 = UUID.randomUUID().toString(); String file1P1C0 = UUID.randomUUID().toString(); - String commitTs = HoodieActiveTimeline.formatDate(Date.from(curDateTime.minusMinutes(minutesForCommit).toInstant())); + String commitTs = TimelineUtils.formatDate(Date.from(curDateTime.minusMinutes(minutesForCommit).toInstant())); try (HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(storageConf, config, context)) { Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { { @@ -427,13 +433,13 @@ private HoodieInstant commitWithMdt(String instantTime, Map metadataWriter.performTableServices(Option.of(instantTime)); metadataWriter.updateFromWriteStatuses(commitMeta, context.emptyHoodieData(), instantTime); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), - serializeCommitMetadata(commitMeta)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMeta)); } else { commitMeta = generateCommitMetadata(instantTime, new HashMap<>()); } metaClient = HoodieTableMetaClient.reload(metaClient); - return new HoodieInstant( + return INSTANT_GENERATOR.createNewInstant( isComplete ? State.COMPLETED : State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime); } @@ -627,7 +633,7 @@ public void testCompactionWithCorruptVersionFile() throws Exception { LSMTimeline.getVersionFilePath(metaClient), Option.of(getUTF8Bytes("invalid_version"))); // check that invalid manifest file will not block archived timeline loading. - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); + HoodieActiveTimeline rawActiveTimeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); assertEquals(5 * 3 + 4, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); @@ -640,7 +646,7 @@ public void testCompactionWithCorruptVersionFile() throws Exception { } // loading archived timeline and active timeline success - HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false); + HoodieActiveTimeline rawActiveTimeline1 = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload(); // check instant number @@ -677,7 +683,7 @@ public void testCompactionRecoverWithoutManifestFile() throws Exception { timelineWriter.compactFiles(candidateFiles, compactedFileName); // check loading archived and active timeline success - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); + HoodieActiveTimeline rawActiveTimeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); assertEquals(5 * 3 + 4, rawActiveTimeline.countInstants() + archivedTimeLine.reload().countInstants()); } @@ -695,7 +701,7 @@ public void testCompactionCleaning() throws Exception { // now we have version 6, 7, 8, 9 version of snapshots // loading archived timeline and active timeline success - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); + HoodieActiveTimeline rawActiveTimeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline(); assertEquals(4 * 3 + 14, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); @@ -749,10 +755,10 @@ public void testArchivalWithMultiWriters(boolean enableMetadata) throws Exceptio throw new HoodieException("Should not have thrown InterruptedException ", e); } metaClient.reloadActiveTimeline(); - while (!metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp().equals(lastInstant.get()) + while (!metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant().get().requestedTime().equals(lastInstant.get()) || metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() > 5) { try { - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(writeConfig, table); archiver.archiveIfRequired(context, true); // if not for below sleep, both archiving threads acquires lock in quick succession and does not give space for main thread // to complete the write operation when metadata table is enabled. @@ -874,7 +880,7 @@ public void testArchiveCommitSavepointNoHole(boolean enableMetadataTable, boolea HoodieTestDataGenerator.createCommitFile(basePath, "104", storageConf); HoodieTestDataGenerator.createCommitFile(basePath, "105", storageConf); HoodieTable table = HoodieSparkTable.create(cfg, context); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); if (enableMetadataTable) { // Simulate a compaction commit in metadata table timeline @@ -891,24 +897,24 @@ public void testArchiveCommitSavepointNoHole(boolean enableMetadataTable, boolea // commits in active timeline = 101 and 105. assertEquals(2, timeline.countInstants(), "Since archiveBeyondSavepoint config is enabled, we will archive commits 102, 103 "); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "101")), "Savepointed commits should always be safe"); - assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")), + assertFalse(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "102")), "102 expected to be archived"); - assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")), + assertFalse(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "103")), "103 expected to be archived"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "105")), "104 expected to be archived"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "105")), "105 expected to be in active timeline"); } else { assertEquals(5, timeline.countInstants(), "Since we have a savepoint at 101, we should never archive any commit after 101 (we only archive 100)"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "101")), "Archived commits should always be safe"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "102")), "Archived commits should always be safe"); - assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")), + assertTrue(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "103")), "Archived commits should always be safe"); } } @@ -972,9 +978,9 @@ public void testArchiveRollbacksTestTable(boolean enableMetadata) throws Excepti if ((originalCommits.size() - commitsAfterArchival.size()) > 0) { hasArchivedInstants = true; List expectedArchivedInstants = instants.subList(0, numArchivedInstants).stream() - .map(p -> new HoodieInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); + .map(p -> INSTANT_GENERATOR.createNewInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); List expectedActiveInstants = instants.subList(numArchivedInstants, instants.size()).stream() - .map(p -> new HoodieInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); + .map(p -> INSTANT_GENERATOR.createNewInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); verifyArchival(expectedArchivedInstants, expectedActiveInstants, commitsAfterArchival, false); } } @@ -1018,13 +1024,13 @@ public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) for (int j = 1; j <= 7; j++) { if (j == 1) { // first commit should not be archived - assertTrue(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); + assertTrue(commitsAfterArchival.contains(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); } else if (j == 2) { // 2nd compaction should not be archived - assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "0000000" + j))); + assertFalse(commitsAfterArchival.contains(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "0000000" + j))); } else { // every other commit should not be archived - assertTrue(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); + assertTrue(commitsAfterArchival.contains(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j))); } } } @@ -1042,7 +1048,7 @@ public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) List commitsAfterArchival = commitsList.getValue(); List archivedInstants = getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003", "00000004"), HoodieTimeline.DELTA_COMMIT_ACTION); - archivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000002")); + archivedInstants.add(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000002")); verifyArchival(archivedInstants, getActiveCommitInstants(Arrays.asList("00000005", "00000006", "00000007", "00000008"), HoodieTimeline.DELTA_COMMIT_ACTION), commitsAfterArchival, false); @@ -1065,13 +1071,13 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTestDataGenerator.createCommitFile(basePath, "1", storageConf); - HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); HoodieTestDataGenerator.createCommitFile(basePath, "2", storageConf); StoragePath markerPath = new StoragePath(metaClient.getMarkerFolderPath("2")); storage.createDirectory(markerPath); - HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); HoodieTestDataGenerator.createCommitFile(basePath, "3", storageConf); - HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); //add 2 more instants to pass filter criteria set in compaction config above HoodieTestDataGenerator.createCommitFile(basePath, "4", storageConf); @@ -1084,7 +1090,7 @@ public void testArchiveCommitTimeline(boolean enableMetadataTable) throws Except } HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context); HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); List archivedInstants = Arrays.asList(instant1, instant2, instant3); @@ -1283,9 +1289,9 @@ public void testArchiveRollbacksAndCleanTestTable() throws Exception { assertThat("The archived commits number is not as expected", allCommits.size() - commitsAfterArchival.size(), is(archived)); List expectedArchiveInstants = instants.subList(0, archived).stream() - .map(p -> new HoodieInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); + .map(p -> INSTANT_GENERATOR.createNewInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); List expectedActiveInstants = instants.subList(archived, instants.size()).stream() - .map(p -> new HoodieInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); + .map(p -> INSTANT_GENERATOR.createNewInstant(State.COMPLETED, p.getValue(), p.getKey())).collect(Collectors.toList()); verifyArchival(expectedArchiveInstants, expectedActiveInstants, commitsAfterArchival, false); } @@ -1310,12 +1316,12 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable List expectedArchivedInstants = new ArrayList<>(); for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant++) { createCleanMetadata(String.format("%02d", startInstant), false, false, isEmpty || i % 2 == 0); - expectedArchivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, String.format("%02d", startInstant))); + expectedArchivedInstants.add(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, String.format("%02d", startInstant))); } for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant += 2) { createCommitAndRollbackFile(startInstant + 1 + "", startInstant + "", false, isEmpty || i % 2 == 0); - expectedArchivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, String.format("%02d", startInstant))); + expectedArchivedInstants.add(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, String.format("%02d", startInstant))); } // Clean and rollback instants are archived only till the last clean instant in the timeline @@ -1328,7 +1334,7 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable } HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context); @@ -1348,7 +1354,7 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable metaClient.getArchivedTimeline().loadCompletedInstantDetailsInMemory(); HoodieInstant firstInstant = metaClient.reloadActiveTimeline().firstInstant().get(); expectedArchivedInstants = expectedArchivedInstants.stream() - .filter(entry -> HoodieTimeline.compareTimestamps(entry.getTimestamp(), HoodieTimeline.LESSER_THAN, firstInstant.getTimestamp() + .filter(entry -> compareTimestamps(entry.requestedTime(), LESSER_THAN, firstInstant.requestedTime() )).collect(Collectors.toList()); expectedArchivedInstants.forEach(entry -> assertTrue(metaClient.getArchivedTimeline().containsInstant(entry))); } @@ -1460,10 +1466,10 @@ public void testArchivalWithMaxDeltaCommitsGuaranteeForCompaction(boolean enable } else { assertEquals(1, originalCommits.size() - commitsAfterArchival.size()); assertFalse(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(0)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(0)))); IntStream.range(2, 10).forEach(j -> assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } } @@ -1486,24 +1492,24 @@ public void testArchivalWithMaxDeltaCommitsGuaranteeForCompaction(boolean enable // first 7 delta commits before the completed compaction should be archived in data table IntStream.range(1, 8).forEach(j -> assertFalse(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); assertEquals(i == 1 ? 6 : 0, originalCommits.size() - commitsAfterArchival.size()); // instant from 11 should be in the active timeline assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(7)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(7)))); assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(8)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(8)))); assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, compactionInstant))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, compactionInstant))); for (int j = 1; j <= i; j++) { assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(9 + j)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(9 + j)))); } } else { // first 9 delta commits before the completed compaction should be archived in data table IntStream.range(1, 10).forEach(j -> assertFalse(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); if (i == 3) { assertEquals(2, originalCommits.size() - commitsAfterArchival.size()); } else if (i < 8) { @@ -1511,16 +1517,16 @@ public void testArchivalWithMaxDeltaCommitsGuaranteeForCompaction(boolean enable } else { assertEquals(1, originalCommits.size() - commitsAfterArchival.size()); assertFalse(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, compactionInstant))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, compactionInstant))); // i == 8 -> [11, 18] should be in the active timeline // i == 9 -> [12, 19] should be in the active timeline if (i == 9) { assertFalse(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(10)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(10)))); } IntStream.range(i - 7, i + 1).forEach(j -> assertTrue(commitsAfterArchival.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(9 + j))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(9 + j))))); } } } @@ -1570,7 +1576,7 @@ public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Excepti // Run archival HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); archiver.archiveIfRequired(context); expectedInstants.remove("1000"); expectedInstants.remove("1001"); @@ -1583,12 +1589,12 @@ public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Excepti HoodieTimeline finalTimeline = timeline; assertEquals(12, expectedInstants.stream().filter(instant -> finalTimeline.containsInstant(instant)).count()); - assertEquals("1002", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); + assertEquals("1002", timeline.getInstantsAsStream().findFirst().get().requestedTime()); // Delete replacecommit requested instant. StoragePath replaceCommitRequestedPath = new StoragePath( basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeRequestedReplaceFileName(replaceInstant)); + + INSTANT_FILE_NAME_GENERATOR.makeRequestedReplaceFileName(replaceInstant)); metaClient.getStorage().deleteDirectory(replaceCommitRequestedPath); metaClient.reloadActiveTimeline(); @@ -1603,7 +1609,7 @@ public void testGetCommitInstantsToArchiveDuringInflightCommits() throws Excepti assertEquals(expectedInstants.size(), timeline.countInstants(), "After archival only first 2 commits should be archived"); HoodieTimeline refreshedTimeline = timeline; assertEquals(8, expectedInstants.stream().filter(instant -> refreshedTimeline.containsInstant(instant)).count()); - assertEquals("1006", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); + assertEquals("1006", timeline.getInstantsAsStream().findFirst().get().requestedTime()); } /** @@ -1623,7 +1629,7 @@ public void testWithOldestReplaceCommit() throws Exception { } HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(cfg, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(cfg, table); HoodieTimeline timeline = metaClient.reloadActiveTimeline(); assertEquals(9, timeline.countInstants(), "Loaded 9 commits and the count should match"); @@ -1631,7 +1637,7 @@ public void testWithOldestReplaceCommit() throws Exception { timeline = metaClient.reloadActiveTimeline(); assertEquals(9, timeline.countInstants(), "Since we have a pending replacecommit at 1001, we should never archive any commit after 1001"); - assertEquals("1001", timeline.getInstantsAsStream().findFirst().get().getTimestamp()); + assertEquals("1001", timeline.getInstantsAsStream().findFirst().get().requestedTime()); } @Test @@ -1678,15 +1684,15 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { // In the metadata table timeline, the first delta commit is "00000000000000000" assertEquals(i + 1, metadataTableInstants.size()); assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(0)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(0)))); } else if (i == 2) { assertEquals(i - 1, metadataTableInstants.size()); assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); assertFalse(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(1)))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(1)))); } else if (i <= 9) { // In the metadata table timeline, the first delta commit is "00000000000000000" // from metadata table init, delta commits 1 till 8 are added @@ -1694,11 +1700,11 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { // rollback in DT will also trigger rollback in MDT assertEquals(i - 1, metadataTableInstants.size()); assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, mdtInitCommit))); // rolled back commits may not be present in MDT timeline [1] IntStream.range(3, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } else if (i == 10) { // i == 10 // The instant "00000000000000000" was archived since it's less than @@ -1709,7 +1715,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { // mdt timeline 3,..., 10, a completed compaction commit IntStream.range(3, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } else if (i <= 13) { // In the metadata table timeline, the first delta commit is 7 // because it equals with the earliest commit on the dataset timeline, after archival, @@ -1719,7 +1725,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { assertEquals(1, metadataTableMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants()); IntStream.range(7, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } else if (i <= 17) { // In the metadata table timeline, the second commit is a compaction commit // from metadata table compaction, after archival, delta commits 14 @@ -1729,7 +1735,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { assertEquals(1, metadataTableMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants()); IntStream.range(10, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } else if (i == 18) { // i == 18 // commits in MDT [10, a completed compaction commit, 11, ... 17, 18, a completed compaction commit] @@ -1738,7 +1744,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { assertEquals(2, metadataTableMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().countInstants()); IntStream.range(10, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } else { // i == 19 // compaction happened in last commit, and archival is triggered with latest compaction retained plus maxInstantToKeep = 6 @@ -1747,7 +1753,7 @@ public void testArchivalAndCompactionInMetadataTable() throws Exception { assertTrue(metadata(writeConfig, context).getLatestCompactionTime().isPresent()); IntStream.range(15, i).forEach(j -> assertTrue(metadataTableInstants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(j - 1))))); } } } @@ -1781,7 +1787,7 @@ public void testRetryArchivalAfterPreviousFailedDeletion() throws Exception { testTable.doWriteOperation("10" + i, WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), 1); } HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(writeConfig, table); HoodieTimeline timeline = metaClient.getActiveTimeline().getWriteTimeline(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); @@ -1797,7 +1803,7 @@ public void testRetryArchivalAfterPreviousFailedDeletion() throws Exception { // Re-running archival again should archive and delete the 101.commit, 102.commit, and 103.commit instant files table.getMetaClient().reloadActiveTimeline(); table = HoodieSparkTable.create(writeConfig, context, metaClient); - archiver = new HoodieTimelineArchiver(writeConfig, table); + archiver = new TimelineArchiverV2(writeConfig, table); assertTrue(archiver.archiveIfRequired(context) > 0); timeline = metaClient.getActiveTimeline().reload().getWriteTimeline(); assertEquals(2, timeline.countInstants(), "The instants from prior archival should " @@ -1819,7 +1825,7 @@ private Pair, List> archiveAndGetCommitsList( : metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants(); List originalCommits = timeline.getInstants(); HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); + TimelineArchiverV2 archiver = new TimelineArchiverV2(writeConfig, table); archiver.archiveIfRequired(context); timeline = includeIncompleteInstants ? metaClient.getActiveTimeline().reload().getAllCommitsTimeline() @@ -1829,14 +1835,14 @@ private Pair, List> archiveAndGetCommitsList( } private void verifyArchival(List expectedArchivedInstants, List expectedActiveInstants, List commitsAfterArchival, boolean isArchivalBeyondSavepoint) { - expectedActiveInstants.sort(Comparator.comparing(HoodieInstant::getTimestamp)); - commitsAfterArchival.sort(Comparator.comparing(HoodieInstant::getTimestamp)); + expectedActiveInstants.sort(Comparator.comparing(HoodieInstant::requestedTime)); + commitsAfterArchival.sort(Comparator.comparing(HoodieInstant::requestedTime)); assertEquals(expectedActiveInstants, commitsAfterArchival); expectedArchivedInstants.forEach(entry -> assertFalse(commitsAfterArchival.contains(entry))); - HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient); + HoodieArchivedTimeline archivedTimeline = TIMELINE_FACTORY.createArchivedTimeline(metaClient); List actualArchivedInstants = archivedTimeline.getInstants(); - actualArchivedInstants.sort(Comparator.comparing(HoodieInstant::getTimestamp)); - expectedArchivedInstants.sort(Comparator.comparing(HoodieInstant::getTimestamp)); + actualArchivedInstants.sort(Comparator.comparing(HoodieInstant::requestedTime)); + expectedArchivedInstants.sort(Comparator.comparing(HoodieInstant::requestedTime)); assertEquals(actualArchivedInstants, expectedArchivedInstants); HoodieTimeline timeline = metaClient.getActiveTimeline(); @@ -1849,7 +1855,7 @@ private void verifyArchival(List expectedArchivedInstants, List getAllArchivedCommitInstants(List commitTime private List getAllArchivedCommitInstants(List commitTimes, String action) { List allInstants = new ArrayList<>(); - commitTimes.forEach(commitTime -> allInstants.add(new HoodieInstant(State.COMPLETED, action, commitTime))); + commitTimes.forEach(commitTime -> allInstants.add(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, action, commitTime))); return allInstants; } @@ -1875,7 +1881,7 @@ private List getActiveSavepointedCommitInstants(List comm private List getActiveCommitInstants(List commitTimes, String action) { List allInstants = new ArrayList<>(); - commitTimes.forEach(entry -> allInstants.add(new HoodieInstant(State.COMPLETED, action, entry))); + commitTimes.forEach(entry -> allInstants.add(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, action, entry))); return allInstants; } @@ -1904,7 +1910,7 @@ private HoodieInstant createRollbackMetadata(String rollbackTime, String commitT HoodieTestTable.of(metaClient).addRollback(rollbackTime, hoodieRollbackMetadata, isEmpty, null); HoodieTestTable.of(metaClient).addRollbackCompleted(rollbackTime, hoodieRollbackMetadata, isEmpty); } - return new HoodieInstant(inflight, "rollback", rollbackTime); + return INSTANT_GENERATOR.createNewInstant(inflight ? State.INFLIGHT : State.COMPLETED, "rollback", rollbackTime); } private void assertInstantListEquals(List expected, List actual) { @@ -1912,7 +1918,7 @@ private void assertInstantListEquals(List expected, List> insertFirstBigBatchForClientCle assertNoWriteErrors(statuses.collect()); // verify that there is a commit metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + HoodieTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting a single commit."); // Should have 100 records in table (check using Index), all in locations marked at commit HoodieTable table = HoodieSparkTable.create(client.getConfig(), context, metaClient); @@ -918,9 +923,9 @@ public void testCleanMarkerDataFilesOnRollback() throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = HoodieSparkTable.create(config, context, metaClient); table.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"), Option.empty()); + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"), Option.empty()); metaClient.reloadActiveTimeline(); - HoodieInstant rollbackInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"); + HoodieInstant rollbackInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"); table.scheduleRollback(context, "002", rollbackInstant, false, config.shouldRollbackUsingMarkers(), false); table.rollback(context, "002", rollbackInstant, true, false); final int numTempFilesAfter = testTable.listAllFilesInTempFolder().length; @@ -1016,8 +1021,8 @@ public void testCleanPreviousCorruptedCleanFiles() throws IOException { String commitTime = HoodieTestTable.makeNewCommitTime(1, "%09d"); List cleanerFileNames = Arrays.asList( - HoodieTimeline.makeRequestedCleanerFileName(commitTime), - HoodieTimeline.makeInflightCleanerFileName(commitTime)); + INSTANT_FILE_NAME_GENERATOR.makeRequestedCleanerFileName(commitTime), + INSTANT_FILE_NAME_GENERATOR.makeInflightCleanerFileName(commitTime)); for (String f : cleanerFileNames) { StoragePath commitFile = new StoragePath(Paths .get(metaClient.getBasePath().toString(), HoodieTableMetaClient.METAFOLDER_NAME, f).toString()); @@ -1170,7 +1175,7 @@ public void testIncrementalFallbackToFullClean() throws Exception { testTable = tearDownTestTableAndReinit(testTable, config); // archive commit 1, 2 - new HoodieTimelineArchiver<>(config, HoodieSparkTable.create(config, context, metaClient)) + new TimelineArchiverV2<>(config, HoodieSparkTable.create(config, context, metaClient)) .archiveIfRequired(context, false); metaClient = HoodieTableMetaClient.reload(metaClient); assertFalse(metaClient.getActiveTimeline().containsInstant("10")); @@ -1321,9 +1326,9 @@ private void testPendingCompactions(HoodieWriteConfig config, int expNumFilesDel .flatMap(cleanStat -> convertPathToFileIdWithCommitTime(newMetaClient, cleanStat.getDeletePathPatterns()) .map(fileIdWithCommitTime -> { if (expFileIdToPendingCompaction.containsKey(fileIdWithCommitTime.getKey())) { - assertTrue(HoodieTimeline.compareTimestamps( + assertTrue(compareTimestamps( fileIdToLatestInstantBeforeCompaction.get(fileIdWithCommitTime.getKey()), - HoodieTimeline.GREATER_THAN, fileIdWithCommitTime.getValue()), + GREATER_THAN, fileIdWithCommitTime.getValue()), "Deleted instant time must be less than pending compaction"); return true; } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index d66ad62ccf14..6f7cf3e4853e 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -35,7 +35,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.Transformations; @@ -83,6 +82,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.apache.hudi.testutils.HoodieSparkClientTestHarness.buildProfile; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -161,7 +163,7 @@ public void testUpsertPartitioner(boolean populateMetaFields) throws Exception { Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("001", deltaCommit.get().getTimestamp(), "Delta commit should be 001"); + assertEquals("001", deltaCommit.get().requestedTime(), "Delta commit should be 001"); Option commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); @@ -195,7 +197,7 @@ public void testUpsertPartitioner(boolean populateMetaFields) throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("002", deltaCommit.get().getTimestamp(), "Latest Delta commit should be 002"); + assertEquals("002", deltaCommit.get().requestedTime(), "Latest Delta commit should be 002"); commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); @@ -285,8 +287,7 @@ public void testLogFileCountsAfterCompaction() throws Exception { table = HoodieSparkTable.create(config, context(), metaClient); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - assertTrue(HoodieTimeline - .compareTimestamps(timeline.lastInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, newCommitTime), + assertTrue(compareTimestamps(timeline.lastInstant().get().requestedTime(), GREATER_THAN, newCommitTime), "Compaction commit should be > than last insert"); for (String partitionPath : dataGen.getPartitionPaths()) { @@ -398,8 +399,7 @@ public void testLogBlocksCountsAfterLogCompaction(boolean populateMetaFields, St table = HoodieSparkTable.create(config, context(), metaClient); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - assertTrue(HoodieTimeline - .compareTimestamps(timeline.lastInstant().get().getTimestamp(), HoodieTimeline.GREATER_THAN, newCommitTime), + assertTrue(compareTimestamps(timeline.lastInstant().get().requestedTime(), GREATER_THAN, newCommitTime), "Compaction commit should be > than last insert"); for (String partitionPath : dataGen.getPartitionPaths()) { @@ -438,10 +438,10 @@ public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Excep String commitActionType = table.getMetaClient().getCommitActionType(); List instants = new ArrayList<>(); String instant0 = metaClient.createNewInstantTime(); - HoodieInstant instant = new HoodieInstant(State.REQUESTED, commitActionType, instant0); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, commitActionType, instant0); activeTimeline.createNewInstant(instant); activeTimeline.transitionRequestedToInflight(instant, Option.empty()); - instant = new HoodieInstant(State.INFLIGHT, commitActionType, instant0); + instant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, commitActionType, instant0); activeTimeline.saveAsComplete(instant, Option.empty()); String instant1 = metaClient.createNewInstantTime(); @@ -455,9 +455,9 @@ public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Excep // Read from commit file table = HoodieSparkTable.create(cfg, context()); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline().getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), - HoodieCommitMetadata.class); + HoodieInstant instantOne = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instantOne, + table.getActiveTimeline().getInstantDetails(instantOne).get(), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { for (HoodieWriteStat stat : pstat.getValue()) { @@ -488,10 +488,9 @@ public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Excep // Read from commit file table = HoodieSparkTable.create(cfg, context()); - metadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline() - .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), - HoodieCommitMetadata.class); + HoodieInstant instant3 = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); + metadata = metaClient.getCommitMetadataSerDe().deserialize(instant3, + table.getActiveTimeline().getInstantDetails(instant3).get(), HoodieCommitMetadata.class); inserts = 0; upserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -529,9 +528,9 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { // Read from commit file HoodieTable table = HoodieSparkTable.create(cfg, context()); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline() - .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), + HoodieInstant instantOne = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instantOne, + table.getActiveTimeline().getInstantDetails(instantOne).get(), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -554,9 +553,9 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { // Read from commit file table = HoodieSparkTable.create(cfg, context()); - metadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline() - .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), + HoodieInstant instantTwo = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); + metadata = metaClient.getCommitMetadataSerDe().deserialize(instantTwo, + table.getActiveTimeline().getInstantDetails(instantTwo).get(), HoodieCommitMetadata.class); inserts = 0; int upserts = 0; @@ -581,9 +580,9 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { // Read from commit file table = HoodieSparkTable.create(cfg, context()); - HoodieCommitMetadata metadata1 = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline() - .getInstantDetails(table.getActiveTimeline().getCommitsTimeline().lastInstant().get()).get(), + HoodieInstant instantThree = table.getActiveTimeline().getCommitsTimeline().lastInstant().get(); + HoodieCommitMetadata metadata1 = metaClient.getCommitMetadataSerDe().deserialize(instantThree, + table.getActiveTimeline().getInstantDetails(instantThree).get(), HoodieCommitMetadata.class); // Ensure that the metadata stats from the extra metadata of delta commits is copied over to the compaction commit @@ -605,9 +604,9 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { // Read from commit file table = HoodieSparkTable.create(cfg, context()); - metadata = HoodieCommitMetadata.fromBytes( - table.getActiveTimeline() - .getInstantDetails(table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get()).get(), + HoodieInstant instant = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); + metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, + table.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); inserts = 0; upserts = 0; @@ -651,7 +650,7 @@ public void testHandleUpdateWithMultiplePartitions() throws Exception { Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("001", deltaCommit.get().getTimestamp(), "Delta commit should be 001"); + assertEquals("001", deltaCommit.get().requestedTime(), "Delta commit should be 001"); Option commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); 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 084948aaac75..2515d4610685 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 @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.util.Option; @@ -58,6 +59,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.model.HoodieCleaningPolicy.KEEP_LATEST_COMMITS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.apache.hudi.table.TestCleaner.insertFirstBigBatchForClientCleanerTest; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; @@ -186,7 +189,7 @@ private void validateFilesAfterCleaning( HoodieInstant lastInstant = commitsTimeline.lastInstant().get(); if (isAsyncClean) { - commitsTimeline = commitsTimeline.findInstantsBefore(lastInstant.getTimestamp()); + commitsTimeline = commitsTimeline.findInstantsBefore(lastInstant.requestedTime()); } // This corresponds to the `earliestCommitToRetain` in {@code CleanPlanner::getFilesToCleanKeepingLatestCommits} Option earliestRetainedCommit = commitsTimeline.nthFromLastInstant(maxCommitsToRetain - 1); @@ -208,13 +211,12 @@ private void validateFilesAfterCleaning( // the latest version before earliestCommitToRetain, which is also kept from cleaning. // The timeline of commits is traversed in reverse order to achieve this. for (HoodieInstant instant : commitsTimeline.getReverseOrderedInstants().collect(Collectors.toList())) { - List hoodieWriteStatList = commitWriteStatsMap.computeIfAbsent(instant.getTimestamp(), newInstant -> { + TimelineLayout layout = TimelineLayout.fromVersion(commitsTimeline.getTimelineLayoutVersion()); + List hoodieWriteStatList = commitWriteStatsMap.computeIfAbsent(instant.requestedTime(), newInstant -> { try { - return HoodieCommitMetadata.fromBytes( - timeline.getInstantDetails( - timeline.filter(inst -> inst.getTimestamp().equals(newInstant)) - .firstInstant().get()).get(), - HoodieCommitMetadata.class) + HoodieInstant instant1 = timeline.filter(inst -> inst.requestedTime().equals(newInstant)) + .firstInstant().get(); + return layout.getCommitMetadataSerDe().deserialize(instant1, timeline.getInstantDetails(instant1).get(), HoodieCommitMetadata.class) .getWriteStats(); } catch (IOException e) { return Collections.EMPTY_LIST; @@ -224,12 +226,12 @@ private void validateFilesAfterCleaning( Pair partitionFileIdPair = Pair.of(writeStat.getPartitionPath(), writeStat.getFileId()); if (remainingFileGroupSet.contains(partitionFileIdPair)) { if (earliestRetainedCommit.isPresent() - && HoodieTimeline.compareTimestamps( - instant.getTimestamp(), HoodieTimeline.LESSER_THAN, earliestRetainedCommit.get().getTimestamp())) { + && compareTimestamps( + instant.requestedTime(), LESSER_THAN, earliestRetainedCommit.get().requestedTime())) { remainingFileGroupSet.remove(partitionFileIdPair); } expectedInstantTimeMap.computeIfAbsent(partitionFileIdPair, k -> new HashSet<>()) - .add(instant.getTimestamp()); + .add(instant.requestedTime()); } }); if (remainingFileGroupSet.isEmpty()) { @@ -247,7 +249,7 @@ private void validateFilesAfterCleaning( commitTimes.add(value.getCommitTime()); }); if (isAsyncClean) { - commitTimes.remove(lastInstant.getTimestamp()); + commitTimes.remove(lastInstant.requestedTime()); } assertEquals( 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 f0cc4c3c7896..03291e82f536 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 @@ -62,6 +62,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.testutils.HoodieTestTable.makeIncrementalCommitTimes; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.table.TestCleaner.insertFirstBigBatchForClientCleanerTest; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.apache.hudi.testutils.HoodieClientTestBase.wrapRecordsGenFunctionForPreppedCalls; @@ -167,7 +168,7 @@ private void testInsertAndCleanByVersions( List instantTimes = makeIncrementalCommitTimes(9, 1, 10); String compactionTime = instantTimes.get(0); table.getActiveTimeline().saveToCompactionRequested( - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime), TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); instantTimes = instantTimes.subList(1, instantTimes.size()); @@ -192,7 +193,7 @@ private void testInsertAndCleanByVersions( HashMap> fileIdToVersions = new HashMap<>(); for (HoodieInstant entry : timeline.getInstants()) { HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class); + metaClient.getCommitMetadataSerDe().deserialize(entry, timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class); for (HoodieWriteStat wstat : commitMetadata.getWriteStats(partitionPath)) { if (!fileIdToVersions.containsKey(wstat.getFileId())) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestAverageRecordSizeUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestAverageRecordSizeUtils.java index a0a95ffca337..8eda75ff7b41 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestAverageRecordSizeUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestAverageRecordSizeUtils.java @@ -22,10 +22,10 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.versioning.v2.BaseTimelineV2; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; @@ -44,6 +44,8 @@ import static org.apache.hudi.common.model.HoodieFileFormat.HOODIE_LOG; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.config.HoodieCompactionConfig.COPY_ON_WRITE_RECORD_SIZE_ESTIMATE; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.mockito.Mockito.mock; @@ -63,7 +65,7 @@ public class TestAverageRecordSizeUtils { public void testAverageRecordSize(List>> instantSizePairs, long expectedSize) { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath("/tmp") .build(); - HoodieDefaultTimeline commitsTimeline = new HoodieDefaultTimeline(); + HoodieTimeline commitsTimeline = new BaseTimelineV2(); List instants = new ArrayList<>(); instantSizePairs.forEach(entry -> { HoodieInstant hoodieInstant = entry.getKey(); @@ -77,7 +79,7 @@ public void testAverageRecordSize(List>> in }); instants.add(hoodieInstant); try { - when(mockTimeline.getInstantDetails(hoodieInstant)).thenReturn(TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + when(mockTimeline.getInstantDetails(hoodieInstant)).thenReturn(TimelineMetadataUtils.serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata)); } catch (IOException e) { throw new RuntimeException("Should not have failed", e); } @@ -89,7 +91,7 @@ public void testAverageRecordSize(List>> in when(mockTimeline.getReverseOrderedInstants()).then(i -> reverseOrderInstants.stream()); commitsTimeline.setInstants(instants); - assertEquals(expectedSize, AverageRecordSizeUtils.averageBytesPerRecord(mockTimeline, writeConfig)); + assertEquals(expectedSize, AverageRecordSizeUtils.averageBytesPerRecord(mockTimeline, writeConfig, COMMIT_METADATA_SER_DE)); } @Test @@ -98,16 +100,16 @@ public void testErrorHandling() { HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder() .withProps(Collections.singletonMap(COPY_ON_WRITE_RECORD_SIZE_ESTIMATE.key(), String.valueOf(recordSize))) .build(false); - HoodieDefaultTimeline commitsTimeline = new HoodieDefaultTimeline(); + BaseTimelineV2 commitsTimeline = new BaseTimelineV2(); List instants = Collections.singletonList( - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1")); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1")); when(mockTimeline.getInstants()).thenReturn(instants); when(mockTimeline.getReverseOrderedInstants()).then(i -> instants.stream()); // Simulate a case where the instant details are absent commitsTimeline.setInstants(new ArrayList<>()); - assertEquals(recordSize, AverageRecordSizeUtils.averageBytesPerRecord(mockTimeline, writeConfig)); + assertEquals(recordSize, AverageRecordSizeUtils.averageBytesPerRecord(mockTimeline, writeConfig, COMMIT_METADATA_SER_DE)); } private static String getBaseFileName(String instantTime) { @@ -128,62 +130,62 @@ static Stream testCases() { // COW // straight forward. just 1 instant. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L)))), 100L)); // two instants. latest instant should be honored arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant + 100)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 100)), 10000000L, 200L)))), 200L)); // two instants, while 2nd one is smaller in size so as to not meet the threshold. So, 1st one should be honored arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 100)), 10000L, 200L)))), 100L)); // 2nd instance is replace commit and should be honored. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, Long.toString(baseInstant + 100)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 100)), 10000000L, 200L)))), 200L)); // MOR // for delta commits, only parquet files should be accounted for. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 100)), 10000000L, 200L)))), 200L)); // delta commit has a mix of parquet and log files. only parquet files should be accounted for. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 1000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), Arrays.asList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 100)), 10000000L, 200L), new HWriteStat(getLogFileName(String.valueOf(baseInstant + 100)), 10000000L, 300L)))), 200L)); // 2nd delta commit only has log files. and so we honor 1st delta commit size. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 10000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), Arrays.asList(new HWriteStat(getLogFileName(String.valueOf(baseInstant + 100)), 1000000L, 200L), new HWriteStat(getLogFileName(String.valueOf(baseInstant + 100)), 10000000L, 300L)))), 100L)); // replace commit should be honored. arguments.add(Arguments.of( - Arrays.asList(Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), + Arrays.asList(Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant)), 1000000L, 100L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, Long.toString(baseInstant + 100)), Arrays.asList(new HWriteStat(getLogFileName(String.valueOf(baseInstant + 100)), 1000000L, 200L), new HWriteStat(getLogFileName(String.valueOf(baseInstant + 100)), 1000000L, 300L))), - Pair.of(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, Long.toString(baseInstant)), + Pair.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, Long.toString(baseInstant)), Collections.singletonList(new HWriteStat(getBaseFileName(String.valueOf(baseInstant + 200)), 1000000L, 400L)))), 400L)); return arguments.stream(); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java index f4f45990689f..a0dad423bd7b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/commit/TestUpsertPartitioner.java @@ -64,7 +64,9 @@ import scala.Tuple2; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.generateFakeHoodieWriteStat; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -112,11 +114,11 @@ private UpsertPartitioner getUpsertPartitioner(int smallFileSize, int numInserts private static List setupHoodieInstants() { List instants = new ArrayList<>(); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts1")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts2")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts3")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts4")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts5")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts1")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts2")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts3")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts4")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "ts5")); Collections.reverse(instants); return instants; } @@ -149,17 +151,17 @@ private static HoodieCommitMetadata generateCommitMetadataWith(int totalRecordsW private static LinkedList> generateCommitMetadataList() throws IOException { LinkedList> commits = new LinkedList<>(); // First commit with non zero records and bytes - commits.push(serializeCommitMetadata(generateCommitMetadataWith(2000, 10000))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(2000, 10000))); // Second commit with non zero records and bytes - commits.push(serializeCommitMetadata(generateCommitMetadataWith(1500, 7500))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(1500, 7500))); // Third commit with a small file - commits.push(serializeCommitMetadata(generateCommitMetadataWith(100, 500))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(100, 500))); // Fourth commit with both zero records and zero bytes - commits.push(serializeCommitMetadata(generateCommitMetadataWith(0, 0))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(0, 0))); // Fifth commit with zero records - commits.push(serializeCommitMetadata(generateCommitMetadataWith(0, 1500))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(0, 1500))); // Sixth commit with zero bytes - commits.push(serializeCommitMetadata(generateCommitMetadataWith(2500, 0))); + commits.push(serializeCommitMetadata(COMMIT_METADATA_SER_DE, generateCommitMetadataWith(2500, 0))); return commits; } @@ -174,7 +176,7 @@ public void testAverageBytesPerRecordForNonEmptyCommitTimeLine() throws Exceptio LinkedList> commits = generateCommitMetadataList(); when(commitTimeLine.getInstantDetails(any(HoodieInstant.class))).thenAnswer(invocationOnMock -> commits.pop()); long expectAvgSize = (long) Math.ceil((1.0 * 7500) / 1500); - long actualAvgSize = AverageRecordSizeUtils.averageBytesPerRecord(commitTimeLine, config); + long actualAvgSize = AverageRecordSizeUtils.averageBytesPerRecord(commitTimeLine, config, COMMIT_METADATA_SER_DE); assertEquals(expectAvgSize, actualAvgSize); } @@ -184,7 +186,7 @@ public void testAverageBytesPerRecordForEmptyCommitTimeLine() throws Exception { HoodieWriteConfig config = makeHoodieClientConfigBuilder().build(); when(commitTimeLine.empty()).thenReturn(true); long expectAvgSize = config.getCopyOnWriteRecordSizeEstimate(); - long actualAvgSize = AverageRecordSizeUtils.averageBytesPerRecord(commitTimeLine, config); + long actualAvgSize = AverageRecordSizeUtils.averageBytesPerRecord(commitTimeLine, config, COMMIT_METADATA_SER_DE); assertEquals(expectAvgSize, actualAvgSize); } 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 d3e58e2f2870..144b48fbc255 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 @@ -62,6 +62,7 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -153,10 +154,10 @@ protected List runNextDeltaCommits(SparkRDDWriteClient client, fin protected void moveCompactionFromRequestedToInflight(String compactionInstantTime, HoodieWriteConfig cfg) { HoodieTableMetaClient metaClient = createMetaClient(cfg.getBasePath()); - HoodieInstant compactionInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant compactionInstant = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionInstant); HoodieInstant instant = metaClient.getActiveTimeline().reload().filterPendingCompactionTimeline().getInstantsAsStream() - .filter(in -> in.getTimestamp().equals(compactionInstantTime)).findAny().get(); + .filter(in -> in.requestedTime().equals(compactionInstantTime)).findAny().get(); assertTrue(instant.isInflight(), "Instant must be marked inflight"); } @@ -164,7 +165,7 @@ protected void scheduleCompaction(String compactionInstantTime, SparkRDDWriteCli client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); HoodieTableMetaClient metaClient = createMetaClient(cfg.getBasePath()); HoodieInstant instant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().get(); - assertEquals(compactionInstantTime, instant.getTimestamp(), "Last compaction instant must be the one set"); + assertEquals(compactionInstantTime, instant.requestedTime(), "Last compaction instant must be the one set"); } /** @@ -179,7 +180,7 @@ protected void scheduleCompaction(String compactionInstantTime, SparkRDDWriteCli protected String tryScheduleCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieWriteConfig cfg) { client.scheduleCompactionAtInstant(compactionInstantTime, Option.empty()); HoodieTableMetaClient metaClient = createMetaClient(cfg.getBasePath()); - return metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().map(HoodieInstant::getTimestamp).orElse(null); + return metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().map(HoodieInstant::requestedTime).orElse(null); } protected void scheduleAndExecuteCompaction(String compactionInstantTime, SparkRDDWriteClient client, HoodieTable table, @@ -217,7 +218,7 @@ protected void executeCompaction(String compactionInstantTime, SparkRDDWriteClie // verify that there is a commit table = getHoodieTable(createMetaClient(cfg.getBasePath()), cfg); HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants(); - String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); + String latestCompactionCommitTime = timeline.lastInstant().get().requestedTime(); assertEquals(latestCompactionCommitTime, compactionInstantTime, "Expect compaction instant time to be the latest commit time"); assertEquals(expectedNumRecs, @@ -241,7 +242,7 @@ protected void executeCompactionWithReplacedFiles(String compactionInstantTime, HoodieTimeline timeline = table.getMetaClient().getCommitTimeline().filterCompletedInstants(); // verify compaction commit is visible in timeline assertTrue(timeline.filterCompletedInstants().getInstantsAsStream() - .filter(instant -> compactionInstantTime.equals(instant.getTimestamp())).findFirst().isPresent()); + .filter(instant -> compactionInstantTime.equals(instant.requestedTime())).findFirst().isPresent()); for (String partition: partitions) { table.getSliceView().getLatestFileSlicesBeforeOrOn(partition, compactionInstantTime, true).forEach(fs -> { // verify that all log files are merged @@ -267,11 +268,11 @@ protected List createNextDeltaCommit(String instantTime, List deltaCommit = metaClient.getActiveTimeline().reload().getDeltaCommitTimeline().filterCompletedInstants().lastInstant(); if (skipCommit && !cfg.shouldAutoCommit()) { - assertTrue(deltaCommit.get().getTimestamp().compareTo(instantTime) < 0, + assertTrue(deltaCommit.get().requestedTime().compareTo(instantTime) < 0, "Delta commit should not be latest instant"); } else { assertTrue(deltaCommit.isPresent()); - assertEquals(instantTime, deltaCommit.get().getTimestamp(), "Delta commit should be latest instant"); + assertEquals(instantTime, deltaCommit.get().requestedTime(), "Delta commit should be latest instant"); } return statusList; } 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 186197317759..06bec6205855 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 @@ -45,6 +45,8 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -82,7 +84,7 @@ public void testRollbackForInflightCompaction() throws Exception { HoodieInstant pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime(), "Pending Compaction instant has expected instant time"); assertEquals(State.REQUESTED, pendingCompactionInstant.getState(), "Pending Compaction instant has expected state"); @@ -93,14 +95,14 @@ public void testRollbackForInflightCompaction() throws Exception { HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context, metaClient); hoodieTable.rollbackInflightCompaction( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime)); metaClient.reloadActiveTimeline(); pendingCompactionInstant = metaClient.getCommitsAndCompactionTimeline().filterPendingCompactionTimeline() .getInstantsAsStream().findFirst().get(); assertEquals("compaction", pendingCompactionInstant.getAction()); assertEquals(State.REQUESTED, pendingCompactionInstant.getState()); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp()); + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime()); // We indirectly test for the race condition where a inflight instant was first deleted then created new. Every // time this happens, the pending compaction instant file in Hoodie Meta path becomes an empty file (Note: Hoodie @@ -108,7 +110,7 @@ public void testRollbackForInflightCompaction() throws Exception { // and look at the file size StoragePathInfo pathInfo = metaClient.getStorage() .getPathInfo(new StoragePath(metaClient.getMetaPath(), - pendingCompactionInstant.getFileName())); + INSTANT_FILE_NAME_GENERATOR.getFileName(pendingCompactionInstant))); assertTrue(pathInfo.getLength() > 0); } } @@ -140,11 +142,11 @@ public void testRollbackInflightIngestionWithPendingCompaction() throws Exceptio metaClient = HoodieTestUtils.createMetaClient(storageConf.newInstance(), cfg.getBasePath()); HoodieInstant pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime(), "Pending Compaction instant has expected instant time"); HoodieInstant inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get(); - assertEquals(inflightInstantTime, inflightInstant.getTimestamp(), "inflight instant has expected instant time"); + assertEquals(inflightInstantTime, inflightInstant.requestedTime(), "inflight instant has expected instant time"); // This should rollback client.startCommitWithTime(nextInflightInstantTime); @@ -152,13 +154,13 @@ public void testRollbackInflightIngestionWithPendingCompaction() throws Exceptio // Validate metaClient = HoodieTestUtils.createMetaClient(storageConf.newInstance(), cfg.getBasePath()); inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get(); - assertEquals(inflightInstant.getTimestamp(), nextInflightInstantTime, "inflight instant has expected instant time"); + assertEquals(inflightInstant.requestedTime(), nextInflightInstantTime, "inflight instant has expected instant time"); assertEquals(1, metaClient.getActiveTimeline() .filterPendingExcludingCompaction().countInstants(), "Expect only one inflight instant"); // Expect pending Compaction to be present pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime(), "Pending Compaction instant has expected instant time"); } } @@ -235,7 +237,7 @@ public void testConcurrentCompaction() throws Exception { // validate the compaction plan does not include pending log files. HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan( - metaClient.reloadActiveTimeline().readCompactionPlanAsBytes(HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime)).get()); + metaClient.reloadActiveTimeline().readCompactionPlanAsBytes(INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime)).get()); assertTrue(compactionPlan.getOperations().stream().noneMatch(op -> op.getDeltaFilePaths().stream().anyMatch(deltaFile -> deltaFile.contains(pendingInstantTime))), "compaction plan should not include pending log files"); @@ -266,7 +268,7 @@ public void testScheduleIngestionBeforePendingCompaction() throws Exception { HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(storageConf.newInstance(), cfg.getBasePath()); HoodieInstant pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), "Pending Compaction instant has expected instant time"); + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime(), "Pending Compaction instant has expected instant time"); assertDoesNotThrow(() -> { runNextDeltaCommits(client, readClient, Collections.singletonList(failedInstantTime), records, cfg, false, @@ -298,7 +300,7 @@ public void testScheduleCompactionAfterPendingIngestion() throws Exception { metaClient = HoodieTestUtils.createMetaClient(storageConf.newInstance(), cfg.getBasePath()); HoodieInstant inflightInstant = metaClient.getActiveTimeline().filterPendingExcludingCompaction().firstInstant().get(); - assertEquals(inflightInstantTime, inflightInstant.getTimestamp(), "inflight instant has expected instant time"); + assertEquals(inflightInstantTime, inflightInstant.requestedTime(), "inflight instant has expected instant time"); assertDoesNotThrow(() -> { // Schedule compaction but do not run them @@ -405,7 +407,7 @@ public void testCompactionOnReplacedFiles() throws Exception { metaClient.reloadActiveTimeline(); HoodieInstant pendingCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant().get(); - assertEquals(compactionInstantTime, pendingCompactionInstant.getTimestamp(), "Pending Compaction instant has expected instant time"); + assertEquals(compactionInstantTime, pendingCompactionInstant.requestedTime(), "Pending Compaction instant has expected instant time"); Set fileGroupsBeforeReplace = getAllFileGroups(hoodieTable, dataGen.getPartitionPaths()); // replace by using insertOverwrite diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java index 7cbdf4bbf7f1..29232732c15a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/compact/TestHoodieCompactor.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -66,6 +65,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; @@ -181,7 +181,7 @@ public void testScheduleCompactionWithInflightInstant() { // create one inflight instance. newCommitTime = "102"; writeClient.startCommitWithTime(newCommitTime); - metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + metaClient.getActiveTimeline().transitionRequestedToInflight(INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); // create one compaction instance before exist inflight instance. @@ -209,7 +209,7 @@ public void testNeedCompactionCondition() throws Exception { // commit 3 (inflight) newCommitTime = "102"; writeClient.startCommitWithTime(newCommitTime); - metaClient.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(State.REQUESTED, + metaClient.getActiveTimeline().transitionRequestedToInflight(INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime), Option.empty()); // check that compaction will not be scheduled 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 cc29bbaf1baf..1ef39fe534e5 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 @@ -103,7 +103,7 @@ public void testSuccessfulCompactionBasedOnNumCommits() throws Exception { metaClient = createMetaClient(cfg.getBasePath()); assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); assertEquals(HoodieTimeline.COMMIT_ACTION, metaClient.getActiveTimeline().lastInstant().get().getAction()); - String compactionTime = metaClient.getActiveTimeline().lastInstant().get().getTimestamp(); + String compactionTime = metaClient.getActiveTimeline().lastInstant().get().requestedTime(); assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), HoodieSparkTable.create(cfg, context), compactionTime).doesMarkerDirExist()); } } @@ -270,7 +270,7 @@ public void testCompactionRetryOnFailureBasedOnNumCommits() throws Exception { // Then: 1 delta commit is done, the failed compaction is retried metaClient = createMetaClient(cfg.getBasePath()); assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); - assertEquals(instantTime2, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); + assertEquals(instantTime2, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().requestedTime()); } @Test @@ -308,7 +308,7 @@ public void testCompactionRetryOnFailureBasedOnTime() throws Exception { metaClient = createMetaClient(cfg.getBasePath()); // 2 delta commits at the beginning. 1 compaction, 1 delta commit following it. assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); - assertEquals(instantTime, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); + assertEquals(instantTime, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().requestedTime()); } @Test @@ -345,6 +345,6 @@ public void testCompactionRetryOnFailureBasedOnNumAndTime() throws Exception { // Then: 1 delta commit is done, the failed compaction is retried metaClient = createMetaClient(cfg.getBasePath()); assertEquals(4, metaClient.getActiveTimeline().getWriteTimeline().countInstants()); - assertEquals(instantTime, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().getTimestamp()); + assertEquals(instantTime, metaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().firstInstant().get().requestedTime()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java index 2fb21b00cc05..29beda6564c6 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestCopyOnWriteRollbackActionExecutor.java @@ -66,6 +66,8 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -105,7 +107,7 @@ public void testCopyOnWriteRollbackActionExecutorForFileListingAsGenerateFile() HoodieWriteConfig writeConfig = getConfigBuilder().withRollbackUsingMarkers(false).build(); HoodieTable table = this.getHoodieTable(metaClient, writeConfig); - HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002"); + HoodieInstant needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "002"); String rollbackInstant = "003"; // execute CopyOnWriteRollbackActionExecutor with filelisting mode BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = @@ -186,7 +188,7 @@ public void testListBasedRollbackStrategy() throws Exception { context = new HoodieSparkEngineContext(jsc); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTable table = this.getHoodieTable(metaClient, cfg); - HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "002"); + HoodieInstant needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "002"); String rollbackInstant = "003"; ListingBasedRollbackStrategy rollbackStrategy = new ListingBasedRollbackStrategy(table, context, table.getConfig(), rollbackInstant, false); @@ -266,7 +268,7 @@ public void testRollbackScale() throws Exception { .withBaseFilesInPartition(p3, fileLengths); HoodieTable table = this.getHoodieTable(metaClient, getConfigBuilder().withRollbackUsingMarkers(false).build()); - HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "003"); + HoodieInstant needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"); // Schedule rollback BaseRollbackPlanActionExecutor copyOnWriteRollbackPlanActionExecutor = @@ -339,7 +341,7 @@ private void performRollbackAndValidate(boolean isUsingMarkers, HoodieWriteConfi this.storage.getScheme() + ":" + rollbackMetadata.get(DEFAULT_SECOND_PARTITION_PATH).getSuccessDeleteFiles().get(0)); - assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, commitInstant.getTimestamp()).doesMarkerDirExist()); + assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, commitInstant.requestedTime()).doesMarkerDirExist()); } @Test @@ -380,7 +382,7 @@ public void testRollbackBackup() throws Exception { StoragePath backupDir = new StoragePath( metaClient.getMetaPath(), table.getConfig().getRollbackBackupDirectory()); assertTrue(storage.exists(new StoragePath( - backupDir, needRollBackInstant.getFileName()))); + backupDir, INSTANT_FILE_NAME_GENERATOR.getFileName(needRollBackInstant)))); assertTrue(storage.exists(new StoragePath( backupDir, testTable.getInflightCommitFilePath("002").getName()))); } @@ -409,7 +411,7 @@ public void testRollbackForMultiwriter() throws Exception { .addCommit("004"); HoodieTable table = this.getHoodieTable(metaClient, getConfigBuilder().build()); - HoodieInstant needRollBackInstant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "003"); + HoodieInstant needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "003"); // execute CopyOnWriteRollbackActionExecutor with filelisting mode CopyOnWriteRollbackActionExecutor copyOnWriteRollbackActionExecutor = @@ -435,13 +437,13 @@ public void testRollbackWhenReplaceCommitIsPresent() throws Exception { dataGen = new HoodieTestDataGenerator(new String[]{partitionStr}); writeBatch(writeClient, firstCommit, "000", Option.of(Arrays.asList("000")), "000", numRecords, dataGen::generateInserts, SparkRDDWriteClient::insert, true, numRecords, numRecords, - 1, true); + 1, true, INSTANT_GENERATOR); // Create second commit. String secondCommit = writeClient.createNewInstantTime(); writeBatch(writeClient, secondCommit, firstCommit, Option.of(Arrays.asList(firstCommit)), "000", 100, dataGen::generateInserts, SparkRDDWriteClient::insert, true, 100, 300, - 2, true); + 2, true, INSTANT_GENERATOR); // Create completed clustering commit Properties properties = new Properties(); @@ -459,7 +461,7 @@ public void testRollbackWhenReplaceCommitIsPresent() throws Exception { ClusteringTestUtils.runClusteringOnInstant(clusteringClient, false, false, clusteringInstant1); HoodieTable table = this.getHoodieTable(metaClient, getConfigBuilder().build()); - HoodieInstant needRollBackInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, secondCommit); + HoodieInstant needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, secondCommit); // Schedule rollback String rollbackInstant = writeClient.createNewInstantTime(); @@ -474,7 +476,7 @@ public void testRollbackWhenReplaceCommitIsPresent() throws Exception { assertThrows(HoodieRollbackException.class, copyOnWriteRollbackActionExecutor::execute); // Schedule rollback for incomplete clustering instant. - needRollBackInstant = new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, clusteringInstant1); + needRollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, clusteringInstant1); rollbackInstant = writeClient.createNewInstantTime(); copyOnWriteRollbackPlanActionExecutor = new BaseRollbackPlanActionExecutor(context, table.getConfig(), table, rollbackInstant, needRollBackInstant, false, diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index da43f5b1578f..e38062fa7ad7 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -68,6 +68,7 @@ import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_SECOND_PARTITION_PATH; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.DEFAULT_THIRD_PARTITION_PATH; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -110,7 +111,8 @@ public void testMergeOnReadRollbackActionExecutor(boolean isUsingMarkers) throws HoodieTable table = this.getHoodieTable(metaClient, cfg); //2. rollback - HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); + HoodieInstant rollBackInstant = INSTANT_GENERATOR.createNewInstant(isUsingMarkers ? HoodieInstant.State.INFLIGHT : HoodieInstant.State.COMPLETED, + HoodieTimeline.DELTA_COMMIT_ACTION, "002"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false, cfg.shouldRollbackUsingMarkers(), false); @@ -197,7 +199,8 @@ public void testMergeOnReadRollbackLogCompactActionExecutorWithListingStrategy(b //3. rollback log compact metaClient.reloadActiveTimeline(); - HoodieInstant rollBackInstant = new HoodieInstant(!isComplete, action, "003"); + HoodieInstant rollBackInstant = INSTANT_GENERATOR.createNewInstant(!isComplete ? HoodieInstant.State.INFLIGHT : HoodieInstant.State.COMPLETED, + action, "003"); HoodieTable table = this.getHoodieTable(metaClient, cfg); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = new BaseRollbackPlanActionExecutor(context, cfg, table, "004", rollBackInstant, false, @@ -282,7 +285,8 @@ public void testMergeOnReadRestoreCompactionCommit() throws IOException { HoodieTable table = this.getHoodieTable(metaClient, cfg); //3. rollback the update to partition1 and partition2 - HoodieInstant rollBackInstant = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); + HoodieInstant rollBackInstant = INSTANT_GENERATOR.createNewInstant(isUsingMarkers ? HoodieInstant.State.INFLIGHT : HoodieInstant.State.COMPLETED, + HoodieTimeline.DELTA_COMMIT_ACTION, "002"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false, cfg.shouldRollbackUsingMarkers(), true); @@ -301,7 +305,8 @@ public void testMergeOnReadRestoreCompactionCommit() throws IOException { assertFalse(WriteMarkersFactory.get(cfg.getMarkersType(), table, "002").doesMarkerDirExist()); // rollback 001 as well. this time since its part of the restore, entire file slice should be deleted and not just log files (for partition1 and partition2) - HoodieInstant rollBackInstant1 = new HoodieInstant(isUsingMarkers, HoodieTimeline.DELTA_COMMIT_ACTION, "001"); + HoodieInstant rollBackInstant1 = INSTANT_GENERATOR.createNewInstant(isUsingMarkers ? HoodieInstant.State.INFLIGHT : HoodieInstant.State.COMPLETED, + HoodieTimeline.DELTA_COMMIT_ACTION, "001"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor1 = new BaseRollbackPlanActionExecutor(context, cfg, table, "004", rollBackInstant1, false, cfg.shouldRollbackUsingMarkers(), true); @@ -366,9 +371,10 @@ public void testRollbackForCanIndexLogFile() throws IOException { String generatedFileID = firstPartitionCommit2FileGroups.get(0).getFileGroupId().getFileId(); // check hoodieCommitMeta - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, table.getMetaClient().getCommitTimeline() - .getInstantDetails(new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "001")) + .getInstantDetails(instant) .get(), HoodieCommitMetadata.class); List firstPartitionWriteStat = commitMetadata.getPartitionToWriteStats().get(DEFAULT_FIRST_PARTITION_PATH); @@ -393,9 +399,10 @@ public void testRollbackForCanIndexLogFile() throws IOException { statuses = client.upsert(writeRecords, newCommitTime); client.commit(newCommitTime, statuses); table = this.getHoodieTable(metaClient, cfg); - commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime); + commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant1, table.getMetaClient().getCommitTimeline() - .getInstantDetails(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime)) + .getInstantDetails(instant1) .get(), HoodieCommitMetadata.class); assertTrue(commitMetadata.getPartitionToWriteStats().containsKey(DEFAULT_FIRST_PARTITION_PATH)); @@ -415,7 +422,7 @@ public void testRollbackForCanIndexLogFile() throws IOException { assertEquals(2, hoodieWriteStatOptionList.get(0).getNumInserts()); // Rollback - HoodieInstant rollBackInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); + HoodieInstant rollBackInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"); BaseRollbackPlanActionExecutor mergeOnReadRollbackPlanActionExecutor = new BaseRollbackPlanActionExecutor(context, cfg, table, "003", rollBackInstant, false, cfg.shouldRollbackUsingMarkers(), false); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java index 2104f37e78fb..00f237648ddc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/action/rollback/TestRollbackUtils.java @@ -34,6 +34,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertIterableEquals; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -47,9 +48,9 @@ private StoragePathInfo generateFileStatus(String filePath) { @Test public void testGenerateHeader() { - HoodieInstant hoodieInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "101"); + HoodieInstant hoodieInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "101"); String instantToRollback = "1"; - Map header = RollbackUtils.generateHeader(instantToRollback, hoodieInstant.getTimestamp()); + Map header = RollbackUtils.generateHeader(instantToRollback, hoodieInstant.requestedTime()); Map headerExpect = new HashMap<>(3); headerExpect.put(HoodieLogBlock.HeaderMetadataType.INSTANT_TIME, "101"); headerExpect.put(HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME, "1"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanActionExecutor.java index 38f7b60b5eee..463c453b6f53 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanActionExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanActionExecutor.java @@ -58,6 +58,9 @@ import java.util.Map; import java.util.UUID; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_PARSER; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -130,7 +133,7 @@ void testPartialCleanFailure(CleanFailureType failureType) throws IOException { HoodieActiveTimeline activeTimeline = mock(HoodieActiveTimeline.class); when(metaClient.getActiveTimeline()).thenReturn(activeTimeline); when(mockHoodieTable.getActiveTimeline()).thenReturn(activeTimeline); - HoodieInstant cleanInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "002"); + HoodieInstant cleanInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "002"); HoodieActiveTimeline cleanTimeline = mock(HoodieActiveTimeline.class); when(activeTimeline.getCleanerTimeline()).thenReturn(cleanTimeline); when(cleanTimeline.getInstants()).thenReturn(Collections.singletonList(cleanInstant)); @@ -138,10 +141,13 @@ void testPartialCleanFailure(CleanFailureType failureType) throws IOException { when(activeTimeline.readCleanerInfoAsBytes(cleanInstant)).thenReturn(TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); when(mockHoodieTable.getCleanTimeline()).thenReturn(cleanTimeline); + when(mockHoodieTable.getInstantGenerator()).thenReturn(INSTANT_GENERATOR); + when(mockHoodieTable.getInstantFileNameGenerator()).thenReturn(INSTANT_FILE_NAME_GENERATOR); + when(mockHoodieTable.getInstantFileNameParser()).thenReturn(INSTANT_FILE_NAME_PARSER); HoodieTimeline inflightsAndRequestedTimeline = mock(HoodieTimeline.class); when(cleanTimeline.filterInflightsAndRequested()).thenReturn(inflightsAndRequestedTimeline); when(inflightsAndRequestedTimeline.getInstants()).thenReturn(Collections.singletonList(cleanInstant)); - when(activeTimeline.transitionCleanRequestedToInflight(any(), any())).thenReturn(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "002")); + when(activeTimeline.transitionCleanRequestedToInflight(any(), any())).thenReturn(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "002")); when(mockHoodieTable.getMetadataWriter("002")).thenReturn(Option.empty()); CleanActionExecutor cleanActionExecutor = new CleanActionExecutor(context, config, mockHoodieTable, "002"); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java index 4ed5cad00aec..720379758c16 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanPlanExecutor.java @@ -28,9 +28,9 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -61,6 +61,7 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNull; @@ -240,10 +241,10 @@ public void testKeepLatestCommits( HoodieCommitMetadata commitMetadata = generateCommitMetadata("00000000000011", Collections.singletonMap(p0, CollectionUtils.createImmutableList(file3P0C2))); metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011")); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011")); metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011"), - serializeCommitMetadata(commitMetadata)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "00000000000011"), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); List hoodieCleanStatsFive2 = runCleaner(config, simulateFailureRetry, simulateMetadataFailure, 12, true); HoodieCleanStat cleanStat = getCleanStat(hoodieCleanStatsFive2, p0); @@ -588,9 +589,9 @@ private void testCleanDeletePartition(HoodieCleanConfig cleanConfig) throws Exce .build(); long now = System.currentTimeMillis(); - String commitInstant = HoodieActiveTimeline.formatDate(new Date(now - 49 * 3600 * 1000)); - String deleteInstant1 = HoodieActiveTimeline.formatDate(new Date(now - 48 * 3600 * 1000)); - String deleteInstant2 = HoodieActiveTimeline.formatDate(new Date(now - 24 * 3600 * 1000)); + String commitInstant = TimelineUtils.formatDate(new Date(now - 49 * 3600 * 1000)); + String deleteInstant1 = TimelineUtils.formatDate(new Date(now - 48 * 3600 * 1000)); + String deleteInstant2 = TimelineUtils.formatDate(new Date(now - 24 * 3600 * 1000)); String p1 = "part_1"; String file1P1 = UUID.randomUUID().toString(); @@ -665,7 +666,7 @@ public void testKeepXHoursWithCleaning( Instant instant = Instant.now(); ZonedDateTime commitDateTime = ZonedDateTime.ofInstant(instant, ZoneId.systemDefault()); int minutesForFirstCommit = 180; - String firstCommitTs = HoodieActiveTimeline.formatDate(Date.from(commitDateTime.minusMinutes(minutesForFirstCommit).toInstant())); + String firstCommitTs = TimelineUtils.formatDate(Date.from(commitDateTime.minusMinutes(minutesForFirstCommit).toInstant())); Map> part1ToFileId = Collections.unmodifiableMap(new HashMap>() { { put(p0, CollectionUtils.createImmutableList(file1P0C0)); @@ -684,7 +685,7 @@ public void testKeepXHoursWithCleaning( // make next commit, with 1 insert & 1 update per partition int minutesForSecondCommit = 150; - String secondCommitTs = HoodieActiveTimeline.formatDate(Date.from(commitDateTime.minusMinutes(minutesForSecondCommit).toInstant())); + String secondCommitTs = TimelineUtils.formatDate(Date.from(commitDateTime.minusMinutes(minutesForSecondCommit).toInstant())); Map partitionAndFileId002 = testTable.addInflightCommit(secondCommitTs).getFileIdsWithBaseFilesInPartitions(p0, p1); String file2P0C1 = partitionAndFileId002.get(p0); String file2P1C1 = partitionAndFileId002.get(p1); @@ -700,7 +701,7 @@ public void testKeepXHoursWithCleaning( // make next commit, with 1 insert per partition int minutesForThirdCommit = 90; - String thirdCommitTs = HoodieActiveTimeline.formatDate(Date.from(commitDateTime.minusMinutes(minutesForThirdCommit).toInstant())); + String thirdCommitTs = TimelineUtils.formatDate(Date.from(commitDateTime.minusMinutes(minutesForThirdCommit).toInstant())); Map partitionAndFileId003 = testTable.addInflightCommit(thirdCommitTs).getFileIdsWithBaseFilesInPartitions(p0, p1); String file3P0C1 = partitionAndFileId003.get(p0); String file3P1C1 = partitionAndFileId003.get(p1); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java index 10a7cf86c13d..920dd832312b 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkCopyOnWriteTableArchiveWithReplace.java @@ -57,7 +57,7 @@ public void testDeletePartitionAndArchive(boolean metadataEnabled) throws IOExce HoodieWriteConfig writeConfig = getConfigBuilder(true) .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).build()) .withArchivalConfig(HoodieArchivalConfig.newBuilder().archiveCommitsWith(4, 5).build()) - .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) + .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(metadataEnabled).build()) .build(); try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig); HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(DEFAULT_PARTITION_PATHS)) { @@ -102,4 +102,4 @@ public void testDeletePartitionAndArchive(boolean metadataEnabled) throws IOExce "should only have the 7 records from the 3rd partition."); } } -} +} \ No newline at end of file diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java index 0f61d90eba52..ffb753aff873 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableInsertUpdateDelete.java @@ -87,6 +87,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSimpleSchema; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -151,8 +153,8 @@ public void testSimpleInsertAndUpdate(HoodieFileFormat fileFormat, boolean popul HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); assertEquals(1, timeline.findInstantsAfter("000", Integer.MAX_VALUE).countInstants(), "Expecting a single commit."); - String latestCompactionCommitTime = timeline.lastInstant().get().getTimestamp(); - assertTrue(HoodieTimeline.compareTimestamps("000", HoodieTimeline.LESSER_THAN, latestCompactionCommitTime)); + String latestCompactionCommitTime = timeline.lastInstant().get().requestedTime(); + assertTrue(compareTimestamps("000", LESSER_THAN, latestCompactionCommitTime)); if (cfg.populateMetaFields()) { assertEquals(200, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), @@ -249,7 +251,7 @@ public void testRepeatedRollbackOfCompaction() throws Exception { HoodieInstant rollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get(); String basePathStr = metaClient.getBasePath().toString(); FileCreateUtils.deleteRollbackCommit(basePathStr.substring(basePathStr.indexOf(":") + 1), - rollbackInstant.getTimestamp()); + rollbackInstant.requestedTime()); metaClient.reloadActiveTimeline(); try (SparkRDDWriteClient client1 = getHoodieWriteClient(cfg)) { // trigger compaction again. @@ -257,7 +259,7 @@ public void testRepeatedRollbackOfCompaction() throws Exception { metaClient.reloadActiveTimeline(); // verify that there is no new rollback instant generated HoodieInstant newRollbackInstant = metaClient.getActiveTimeline().getRollbackTimeline().lastInstant().get(); - assertEquals(rollbackInstant.getTimestamp(), newRollbackInstant.getTimestamp()); + assertEquals(rollbackInstant.requestedTime(), newRollbackInstant.requestedTime()); } } } @@ -292,7 +294,7 @@ public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws E Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("001", deltaCommit.get().getTimestamp(), "Delta commit should be 001"); + assertEquals("001", deltaCommit.get().requestedTime(), "Delta commit should be 001"); Option commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); @@ -335,7 +337,7 @@ public void testSimpleInsertUpdateAndDelete(boolean populateMetaFields) throws E metaClient = HoodieTableMetaClient.reload(metaClient); deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("004", deltaCommit.get().getTimestamp(), "Latest Delta commit should be 004"); + assertEquals("004", deltaCommit.get().requestedTime(), "Latest Delta commit should be 004"); commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java index cb3d35081580..dbdb11988f88 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestHoodieSparkMergeOnReadTableRollback.java @@ -77,7 +77,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.junit.jupiter.api.Assertions.assertAll; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -115,7 +119,7 @@ void testCOWToMORConvertedTableRollback(boolean rollbackUsingMarkers) throws Exc metaClient = HoodieTableMetaClient.reload(metaClient); Option commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertTrue(commit.isPresent()); - assertEquals("001", commit.get().getTimestamp(), "commit should be 001"); + assertEquals("001", commit.get().requestedTime(), "commit should be 001"); /* * Write 2 (updates) @@ -187,7 +191,7 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro Option deltaCommit = metaClient.getActiveTimeline().getDeltaCommitTimeline().firstInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals("000000001", deltaCommit.get().getTimestamp(), "Delta commit should be 000000001"); + assertEquals("000000001", deltaCommit.get().requestedTime(), "Delta commit should be 000000001"); Option commit = metaClient.getActiveTimeline().getCommitAndReplaceTimeline().firstInstant(); assertFalse(commit.isPresent()); @@ -317,10 +321,10 @@ void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) thro metaClient = HoodieTableMetaClient.reload(metaClient); - final String compactedCommitTime = metaClient.getActiveTimeline().reload().lastInstant().get().getTimestamp(); + final String compactedCommitTime = metaClient.getActiveTimeline().reload().lastInstant().get().requestedTime(); assertTrue(listAllBaseFilesInPath(hoodieTable).stream() .anyMatch(file -> compactedCommitTime.equals(new HoodieBaseFile(file).getCommitTime()))); - hoodieTable.rollbackInflightCompaction(new HoodieInstant( + hoodieTable.rollbackInflightCompaction(INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactedCommitTime)); allFiles = listAllBaseFilesInPath(hoodieTable); metaClient = HoodieTableMetaClient.reload(metaClient); @@ -372,7 +376,7 @@ void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { HoodieInstant commitInstant = instantCommitMetadataPairOpt.get().getKey(); - assertEquals("001", commitInstant.getTimestamp()); + assertEquals("001", commitInstant.requestedTime()); assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, commitInstant.getAction()); assertEquals(200, getTotalRecordsWritten(instantCommitMetadataPairOpt.get().getValue())); @@ -479,7 +483,7 @@ void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception { tableView = getHoodieTableFileSystemView(metaClient, metaClient.getCommitsTimeline(), allFiles); final String compactedCommitTime = - metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().getTimestamp(); + metaClient.getActiveTimeline().reload().getCommitsTimeline().lastInstant().get().requestedTime(); assertTrue(tableView.getLatestBaseFiles().anyMatch(file -> compactedCommitTime.equals(file.getCommitTime()))); @@ -594,7 +598,7 @@ void testRestoreWithCleanedUpCommits() throws Exception { metaClient.getCommitTimeline().filterCompletedInstants(), allFiles); Stream dataFilesToRead = tableView.getLatestBaseFiles(); assertFalse(dataFilesToRead.anyMatch( - file -> HoodieTimeline.compareTimestamps("002", HoodieTimeline.GREATER_THAN, + file -> compareTimestamps("002", GREATER_THAN, file.getCommitTime()))); client.deleteSavepoint("002"); @@ -788,11 +792,11 @@ void testInsertsGeneratedIntoLogFilesRollback(boolean rollbackUsingMarkers) thro java.nio.file.Path tempFolder = Files.createTempDirectory(this.getClass().getCanonicalName()); Map fileNameMap = new HashMap<>(); for (HoodieInstant.State state : Arrays.asList(HoodieInstant.State.REQUESTED, HoodieInstant.State.INFLIGHT)) { - HoodieInstant toCopy = new HoodieInstant(state, HoodieTimeline.DELTA_COMMIT_ACTION, lastCommitTime); + HoodieInstant toCopy = INSTANT_GENERATOR.createNewInstant(state, HoodieTimeline.DELTA_COMMIT_ACTION, lastCommitTime); File file = Files.createTempFile(tempFolder, null, null).toFile(); - fs().copyToLocalFile(new Path(metaClient.getMetaPath().toString(), toCopy.getFileName()), + fs().copyToLocalFile(new Path(metaClient.getMetaPath().toString(), INSTANT_FILE_NAME_GENERATOR.getFileName(toCopy)), new Path(file.getAbsolutePath())); - fileNameMap.put(file.getAbsolutePath(), toCopy.getFileName()); + fileNameMap.put(file.getAbsolutePath(), INSTANT_FILE_NAME_GENERATOR.getFileName(toCopy)); } Path markerDir = new Path(Files.createTempDirectory(tempFolder, null).toAbsolutePath().toString()); if (rollbackUsingMarkers) { @@ -877,7 +881,7 @@ void testInsertsGeneratedIntoLogFilesRollbackAfterCompaction(boolean rollbackUsi //writeClient.commitCompaction(newCommitTime, statuses, Option.empty()); // Trigger a rollback of compaction table.getActiveTimeline().reload(); - table.rollbackInflightCompaction(new HoodieInstant( + table.rollbackInflightCompaction(INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, newCommitTime)); metaClient = HoodieTableMetaClient.reload(metaClient); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java index 443a82078001..f158d4605318 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestMarkerBasedRollbackStrategy.java @@ -55,6 +55,7 @@ import java.util.List; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -99,7 +100,7 @@ public void testMarkerBasedRollbackAppend() throws Exception { HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), - "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); + "002").getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); assertEquals(1, rollbackRequests.size()); } @@ -117,7 +118,7 @@ public void testMarkerBasedRollbackAppendWithLogFileMarkers(IOType testIOType) t HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), - "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); + "002").getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); assertEquals(1, rollbackRequests.size()); HoodieRollbackRequest rollbackRequest = rollbackRequests.get(0); assertEquals("partA", rollbackRequest.getPartitionPath()); @@ -144,10 +145,10 @@ public void testCopyOnWriteRollbackWithTestTable() throws Exception { // when HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), - "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); + "002").getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); List stats = new BaseRollbackHelper(hoodieTable.getMetaClient(), getConfig()).performRollback(context, - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001"), rollbackRequests); // then: ensure files are deleted correctly, non-existent files reported as failed deletes @@ -223,11 +224,11 @@ private List testInsertAndRollback(SparkRDDWriteClient write HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), - "002").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); + "002").getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001")); // rollback 1st commit and ensure stats reflect the info. return new BaseRollbackHelper(hoodieTable.getMetaClient(), getConfig()).performRollback(context, - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"), rollbackRequests); } @@ -248,11 +249,11 @@ private List testUpdateAndRollback(boolean useFileListingMet HoodieTable hoodieTable = HoodieSparkTable.create(getConfig(), context, metaClient); List rollbackRequests = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), - "003").getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002")); + "003").getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002")); // rollback 2nd commit and ensure stats reflect the info. return new BaseRollbackHelper(hoodieTable.getMetaClient(), getConfig()).performRollback(context, - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), rollbackRequests); } @@ -270,7 +271,7 @@ public void testMarkerBasedRollbackFallbackToTimelineServerWhenDirectMarkerFails initMocks(this); when(writeMarkers.allMarkerFilePaths()).thenThrow(new IOException("Markers.type file not present")); MarkerBasedRollbackStrategy rollbackStrategy = new MarkerBasedRollbackStrategy(hoodieTable, context, getConfig(), "002"); - List rollbackRequests = rollbackStrategy.getRollbackRequests(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); + List rollbackRequests = rollbackStrategy.getRollbackRequests(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "001")); assertEquals(1, rollbackRequests.size()); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java index 61ee4a50aa84..b8f55c55a724 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestSparkNonBlockingConcurrencyControl.java @@ -294,15 +294,15 @@ public void testMultiBaseFile(boolean bulkInsertFirst) throws Exception { if (bulkInsertFirst) { assertEquals(3, instants.size()); // check that bulk insert finished before the upsert started - assertTrue(Long.parseLong(instants.get(0).getCompletionTime()) < Long.parseLong(instants.get(1).getTimestamp())); + assertTrue(Long.parseLong(instants.get(0).getCompletionTime()) < Long.parseLong(instants.get(1).requestedTime())); // check that the upserts overlapped in time - assertTrue(Long.parseLong(instants.get(1).getCompletionTime()) > Long.parseLong(instants.get(2).getTimestamp())); - assertTrue(Long.parseLong(instants.get(2).getCompletionTime()) > Long.parseLong(instants.get(1).getTimestamp())); + assertTrue(Long.parseLong(instants.get(1).getCompletionTime()) > Long.parseLong(instants.get(2).requestedTime())); + assertTrue(Long.parseLong(instants.get(2).getCompletionTime()) > Long.parseLong(instants.get(1).requestedTime())); } else { assertEquals(2, instants.size()); // check that the upserts overlapped in time - assertTrue(Long.parseLong(instants.get(0).getCompletionTime()) > Long.parseLong(instants.get(1).getTimestamp())); - assertTrue(Long.parseLong(instants.get(1).getCompletionTime()) > Long.parseLong(instants.get(0).getTimestamp())); + assertTrue(Long.parseLong(instants.get(0).getCompletionTime()) > Long.parseLong(instants.get(1).requestedTime())); + assertTrue(Long.parseLong(instants.get(1).getCompletionTime()) > Long.parseLong(instants.get(0).requestedTime())); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java index a5fe87767190..98e5ba6a8de0 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/upgrade/TestUpgradeDowngrade.java @@ -215,12 +215,12 @@ public void testUpgradeZeroToOneInternal(boolean induceResiduesFromPrevUpgrade, // delete one of the marker files in 2nd commit if need be. WriteMarkers writeMarkers = - WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitsInstant.getTimestamp()); + WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitsInstant.requestedTime()); List markerPaths = new ArrayList<>(writeMarkers.allMarkerFilePaths()); if (deletePartialMarkerFiles) { String toDeleteMarkerFile = markerPaths.get(0); table.getStorage().deleteDirectory(new StoragePath( - table.getMetaClient().getTempFolderPath() + "/" + commitsInstant.getTimestamp() + table.getMetaClient().getTempFolderPath() + "/" + commitsInstant.requestedTime() + "/" + toDeleteMarkerFile)); markerPaths.remove(toDeleteMarkerFile); } @@ -409,7 +409,7 @@ private void testUpgradeFourToFiveInternal(boolean assertDefaultPartition, boole initMetaClient(getTableType(), properties); // init config, table and client. - HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(true).withRollbackUsingMarkers(false) + HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(true).withRollbackUsingMarkers(false).withWriteTableVersion(6) .doSkipDefaultPartitionValidation(skipDefaultPartitionValidation).withProps(params).build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); // Write inserts @@ -457,6 +457,7 @@ private void addNewTableParamsToProps(Map params, String tableNa params.put(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key(), "partition_path"); params.put(HoodieTableConfig.NAME.key(), tableName); params.put(BASE_FILE_FORMAT.key(), BASE_FILE_FORMAT.defaultValue().name()); + params.put("hoodie.table.version", "6"); } private void doInsert(SparkRDDWriteClient client) { @@ -600,13 +601,14 @@ public void testDowngrade( // init config, table and client. Map params = new HashMap<>(); if (fromVersion.versionCode() >= HoodieTableVersion.TWO.versionCode()) { - addNewTableParamsToProps(params); + addNewTableParamsToProps(params, metaClient.getTableConfig().getTableName()); } if (tableType == HoodieTableType.MERGE_ON_READ) { params.put(TYPE.key(), HoodieTableType.MERGE_ON_READ.name()); metaClient = HoodieTestUtils.init(storageConf, basePath, HoodieTableType.MERGE_ON_READ); } HoodieWriteConfig cfg = getConfigBuilder().withAutoCommit(false).withRollbackUsingMarkers(true) + .withWriteTableVersion(6) .withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadataTable).build()) .withMarkersType(markerType.name()).withProps(params).build(); SparkRDDWriteClient client = getHoodieWriteClient(cfg); @@ -620,6 +622,10 @@ public void testDowngrade( tableConfig.setValue(BASE_FILE_FORMAT, cfg.getString(BASE_FILE_FORMAT)); } + // Downgrade Script for table version 8 is still in progress. + assertTrue(HoodieTableVersion.SEVEN.greaterThan(fromVersion)); + prepForDowngradeFromVersion(HoodieTableVersion.SIX); + // prepare data. Make 2 commits, in which 2nd is not committed. List firstPartitionCommit2FileSlices = new ArrayList<>(); List secondPartitionCommit2FileSlices = new ArrayList<>(); @@ -629,12 +635,12 @@ public void testDowngrade( HoodieInstant commitsInstant = table.getPendingCommitsTimeline().lastInstant().get(); // delete one of the marker files in 2nd commit if need be. - WriteMarkers writeMarkers = WriteMarkersFactory.get(markerType, table, commitsInstant.getTimestamp()); + WriteMarkers writeMarkers = WriteMarkersFactory.get(markerType, table, commitsInstant.requestedTime()); List markerPaths = new ArrayList<>(writeMarkers.allMarkerFilePaths()); if (deletePartialMarkerFiles) { String toDeleteMarkerFile = markerPaths.get(0); table.getStorage().deleteDirectory(new StoragePath( - table.getMetaClient().getTempFolderPath() + "/" + commitsInstant.getTimestamp() + table.getMetaClient().getTempFolderPath() + "/" + commitsInstant.requestedTime() + "/" + toDeleteMarkerFile)); markerPaths.remove(toDeleteMarkerFile); } @@ -667,11 +673,11 @@ public void testDowngrade( private void assertMarkerFilesForDowngrade(HoodieTable table, HoodieInstant commitInstant, boolean assertExists) throws IOException { // Verify recreated marker files are as expected - WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp()); + WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.requestedTime()); if (assertExists) { assertTrue(writeMarkers.doesMarkerDirExist()); assertEquals(0, getTimelineServerBasedMarkerFileCount( - table.getMetaClient().getMarkerFolderPath(commitInstant.getTimestamp()), + table.getMetaClient().getMarkerFolderPath(commitInstant.requestedTime()), (FileSystem) table.getStorage().getFileSystem())); } else { assertFalse(writeMarkers.doesMarkerDirExist()); @@ -690,7 +696,7 @@ private long getTimelineServerBasedMarkerFileCount(String markerDir, FileSystem private void assertMarkerFilesForUpgrade(HoodieTable table, HoodieInstant commitInstant, List firstPartitionCommit2FileSlices, List secondPartitionCommit2FileSlices) throws IOException { // Verify recreated marker files are as expected - WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.getTimestamp()); + WriteMarkers writeMarkers = WriteMarkersFactory.get(getConfig().getMarkersType(), table, commitInstant.requestedTime()); assertTrue(writeMarkers.doesMarkerDirExist()); Set files = writeMarkers.allMarkerFilePaths(); @@ -717,7 +723,7 @@ private void assertMarkerFilesForUpgrade(HoodieTable table, HoodieInstant commit expectedLogFilePaths.add(Pair.of(partitionPath + "/" + logFile.getFileId(), logBaseCommitTime)); } } - if (fileSlice.getBaseInstantTime().equals(commitInstant.getTimestamp())) { + if (fileSlice.getBaseInstantTime().equals(commitInstant.requestedTime())) { String path = fileSlice.getBaseFile().get().getPath(); // for base files, path can be asserted as is. expectedPaths.add(path.substring(path.indexOf(partitionPath))); @@ -882,7 +888,7 @@ private void prepForUpgradeFromZeroToOne(HoodieTable table) throws IOException { .collect(Collectors.toList()); for (HoodieInstant instant : instantsToBeParsed) { WriteMarkers writeMarkers = - WriteMarkersFactory.get(table.getConfig().getMarkersType(), table, instant.getTimestamp()); + WriteMarkersFactory.get(table.getConfig().getMarkersType(), table, instant.requestedTime()); Set oldMarkers = writeMarkers.allMarkerFilePaths(); boolean hasAppendMarker = oldMarkers.stream().anyMatch(marker -> marker.contains(IOType.APPEND.name()) || marker.contains(IOType.CREATE.name())); if (hasAppendMarker) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieCleanerTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieCleanerTestBase.java index 907d9ba9152c..8d12c7da48de 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieCleanerTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieCleanerTestBase.java @@ -53,6 +53,7 @@ import static org.apache.hudi.common.bootstrap.index.TestBootstrapIndex.generateBootstrapIndex; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestTable.makeNewCommitTime; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -117,7 +118,7 @@ protected List runCleaner( } if (simulateRetryFailure) { - HoodieInstant completedCleanInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs); + HoodieInstant completedCleanInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLEAN_ACTION, cleanInstantTs); HoodieCleanMetadata metadata = CleanerUtils.getCleanerMetadata(metaClient, completedCleanInstant); metadata.getPartitionMetadata().values().forEach(p -> { String dirPath = metaClient.getBasePath() + "/" + p.getPartitionPath(); @@ -135,8 +136,8 @@ protected List runCleaner( // Simulate the failure of corresponding instant in the metadata table HoodieTableMetaClient metadataMetaClient = HoodieTestUtils.createMetaClient( metaClient.getStorageConf(), - HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())); - HoodieInstant deltaCommit = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, cleanInstantTs); + HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath()), metaClient.getTableConfig().getTableVersion()); + HoodieInstant deltaCommit = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, cleanInstantTs); metadataMetaClient.reloadActiveTimeline().revertToInflight(deltaCommit); } @@ -159,7 +160,7 @@ protected List runCleaner( .withFailedDeletes(x.getFailedDeleteFiles()).withSuccessfulDeletes(x.getSuccessDeleteFiles()) .withPolicy(HoodieCleaningPolicy.valueOf(x.getPolicy())).withDeletePathPattern(x.getDeletePathPatterns()) .withEarliestCommitRetained(Option.ofNullable(cleanMetadata1.getEarliestCommitToRetain() != null - ? new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "000") + ? INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "000") : null)) .build()) .collect(Collectors.toMap(HoodieCleanStat::getPartitionPath, x -> x)); @@ -169,7 +170,7 @@ protected List runCleaner( .withFailedDeletes(s.getFailedDeleteFiles()).withSuccessfulDeletes(s.getSuccessDeleteFiles()) .withPolicy(HoodieCleaningPolicy.valueOf(x.getPolicy())).withDeletePathPattern(s.getDeletePathPatterns()) .withEarliestCommitRetained(Option.ofNullable(s.getEarliestCommitToRetain()) - .map(y -> new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, y))) + .map(y -> INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, y))) .withSuccessfulDeleteBootstrapBaseFiles(x.getSuccessDeleteFiles()) .withFailedDeleteBootstrapBaseFiles(x.getFailedDeleteFiles()) .withDeleteBootstrapBasePathPatterns(x.getDeletePathPatterns()).build()); @@ -212,8 +213,8 @@ public void commitWithMdt(String instantTime, Map> partToFi metadataWriter.performTableServices(Option.of(instantTime)); metadataWriter.updateFromWriteStatuses(commitMeta, context.emptyHoodieData(), instantTime); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), - serializeCommitMetadata(commitMeta)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMeta)); metaClient = HoodieTableMetaClient.reload(metaClient); } } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java index 107ce6a84de0..bbb5829b5b00 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieClientTestBase.java @@ -28,8 +28,9 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.view.SyncableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.FileFormatUtils; @@ -227,8 +228,9 @@ public Function> generateWrapDeleteKeysFn(boolean isPre public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit) throws Exception { - return insertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, isPreppedAPI, assertForCommit, expRecordsInThisCommit, true); + boolean assertForCommit, int expRecordsInThisCommit, InstantGenerator instantGenerator) throws Exception { + return insertFirstBatch(writeConfig, client, newCommitTime, initCommitTime, numRecordsInThisCommit, writeFn, isPreppedAPI, + assertForCommit, expRecordsInThisCommit, true, instantGenerator); } /** @@ -249,12 +251,13 @@ public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, Spar public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false, filterForCommitTimeWithAssert); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expRecordsInThisCommit, 1, false, + filterForCommitTimeWithAssert, instantGenerator); } /** @@ -277,7 +280,8 @@ public JavaRDD insertFirstBatch(HoodieWriteConfig writeConfig, Spar public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, Option partition) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + Option partition, InstantGenerator instantGenerator) throws Exception { if (partition.isPresent()) { final Function3, String, Integer, String> recordGenFunction = @@ -285,13 +289,13 @@ public JavaRDD insertBatch(HoodieWriteConfig writeConfig, SparkRDDW return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, - partition.get()); + partition.get(), instantGenerator); } else { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateInserts); return writeBatch(client, newCommitTime, initCommitTime, Option.empty(), initCommitTime, numRecordsInThisCommit, - recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false); + recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, false, instantGenerator); } } @@ -299,9 +303,10 @@ public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDW String prevCommitTime, Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, + InstantGenerator instantGenerator) throws Exception { return updateBatch(writeConfig, client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, writeFn, - isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, true); + isPreppedAPI, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, true, instantGenerator); } /** @@ -328,20 +333,20 @@ public JavaRDD updateBatch(HoodieWriteConfig writeConfig, SparkRDDW int numRecordsInThisCommit, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean isPreppedAPI, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { final Function2, String, Integer> recordGenFunction = generateWrapRecordsFn(isPreppedAPI, writeConfig, dataGen::generateUniqueUpdates); return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, false, filterForCommitTimeWithAssert); + expTotalCommits, false, filterForCommitTimeWithAssert, instantGenerator); } public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, boolean assertForCommit, - int expRecordsInThisCommit, int expTotalRecords) throws Exception { + int expRecordsInThisCommit, int expTotalRecords, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { return deleteBatch(writeConfig, client, newCommitTime, prevCommitTime, initCommitTime, numRecordsInThisCommit, isPreppedAPI, - assertForCommit, expRecordsInThisCommit, expTotalRecords, true); + assertForCommit, expRecordsInThisCommit, expTotalRecords, true, timelineFactory, instantGenerator); } /** @@ -362,7 +367,8 @@ public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDW */ public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, String initCommitTime, int numRecordsInThisCommit, boolean isPreppedAPI, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, boolean filterForCommitTimeWithAssert) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + boolean filterForCommitTimeWithAssert, TimelineFactory timelineFactory, InstantGenerator instantGenerator) throws Exception { if (isPreppedAPI) { final Function2, String, Integer> recordGenFunction = @@ -375,7 +381,8 @@ public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDW Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn = SparkRDDWriteClient::deletePrepped; JavaRDD result = deleteFn.apply(client, deleteRecords, newCommitTime); - return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, result); + return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, + filterForCommitTimeWithAssert, result, timelineFactory, instantGenerator); } else { final Function> keyGenFunction = generateWrapDeleteKeysFn(isPreppedAPI, writeConfig, dataGen::generateUniqueDeletes); @@ -391,7 +398,8 @@ public JavaRDD deleteBatch(HoodieWriteConfig writeConfig, SparkRDDW Function3, SparkRDDWriteClient, JavaRDD, String> deleteFn = SparkRDDWriteClient::delete; JavaRDD result = deleteFn.apply(client, deleteRecords, newCommitTime); - return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, filterForCommitTimeWithAssert, result); + return getWriteStatusAndVerifyDeleteOperation(newCommitTime, prevCommitTime, initCommitTime, assertForCommit, expRecordsInThisCommit, expTotalRecords, + filterForCommitTimeWithAssert, result, timelineFactory, instantGenerator); } } @@ -399,9 +407,10 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom Option> commitTimesBetweenPrevAndNew, String initCommitTime, int numRecordsInThisCommit, Function2, String, Integer> recordGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, - boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit) throws Exception { + boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, + int expTotalCommits, boolean doCommit, InstantGenerator instantGenerator) throws Exception { return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, - writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true); + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, instantGenerator); } public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, @@ -409,9 +418,9 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom Function3, String, Integer, String> recordGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, - boolean doCommit, String partition) throws Exception { + boolean doCommit, String partition, InstantGenerator instantGenerator) throws Exception { return writeBatch(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, recordGenFunction, - writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition); + writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, expTotalCommits, doCommit, true, partition, instantGenerator); } /** @@ -437,12 +446,12 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom Function2, String, Integer> recordGenFunction, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, - boolean filterForCommitTimeWithAssert) throws Exception { + boolean filterForCommitTimeWithAssert, InstantGenerator instantGenerator) throws Exception { List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit); return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, doCommit, filterForCommitTimeWithAssert); + expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } public JavaRDD writeBatch(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, @@ -451,12 +460,13 @@ public JavaRDD writeBatch(SparkRDDWriteClient client, String newCom Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert, - String partition) throws Exception { + String partition, + InstantGenerator instantGenerator) throws Exception { List records = recordGenFunction.apply(newCommitTime, numRecordsInThisCommit, partition); return writeBatchHelper(client, newCommitTime, prevCommitTime, commitTimesBetweenPrevAndNew, initCommitTime, numRecordsInThisCommit, records, writeFn, assertForCommit, expRecordsInThisCommit, expTotalRecords, - expTotalCommits, doCommit, filterForCommitTimeWithAssert); + expTotalCommits, doCommit, filterForCommitTimeWithAssert, instantGenerator); } private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String newCommitTime, String prevCommitTime, @@ -464,7 +474,8 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String int numRecordsInThisCommit, List records, Function3, SparkRDDWriteClient, JavaRDD, String> writeFn, boolean assertForCommit, int expRecordsInThisCommit, int expTotalRecords, - int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert) throws IOException { + int expTotalCommits, boolean doCommit, boolean filterForCommitTimeWithAssert, + InstantGenerator instantGenerator) throws IOException { // Write 1 (only inserts) client.startCommitWithTime(newCommitTime); @@ -487,11 +498,11 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String if (assertForCommit) { assertEquals(expTotalCommits, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(), "Expecting " + expTotalCommits + " commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be " + newCommitTime); if (filterForCommitTimeWithAssert) { // when meta cols are disabled, we can't really do per commit assertion. assertEquals(expRecordsInThisCommit, - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, instantGenerator).count(), "Must contain " + expRecordsInThisCommit + " records"); } @@ -508,7 +519,7 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String if (filterForCommitTimeWithAssert) { // Check that the incremental consumption from prevCommitTime assertEquals( - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, instantGenerator).count(), HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(prevCommitTime)), "Incremental consumption from " + prevCommitTime @@ -516,7 +527,7 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String if (commitTimesBetweenPrevAndNew.isPresent()) { commitTimesBetweenPrevAndNew.get().forEach(ct -> { assertEquals( - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime) + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, instantGenerator) .count(), HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(ct)), "Incremental consumption from " + ct + " should give all records in latest commit"); @@ -528,22 +539,23 @@ private JavaRDD writeBatchHelper(SparkRDDWriteClient client, String } private JavaRDD getWriteStatusAndVerifyDeleteOperation(String newCommitTime, String prevCommitTime, String initCommitTime, boolean assertForCommit, int expRecordsInThisCommit, - int expTotalRecords, boolean filerForCommitTimeWithAssert, JavaRDD result) { + int expTotalRecords, boolean filerForCommitTimeWithAssert, JavaRDD result, + TimelineFactory timelineFactory, InstantGenerator instantGenerator) { List statuses = result.collect(); assertNoWriteErrors(statuses); // verify that there is a commit HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient(storageConf, basePath); - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitAndReplaceTimeline(); + HoodieTimeline timeline = timelineFactory.createActiveTimeline(metaClient).getCommitAndReplaceTimeline(); if (assertForCommit) { assertEquals(3, timeline.findInstantsAfter(initCommitTime, Integer.MAX_VALUE).countInstants(), "Expecting 3 commits."); - assertEquals(newCommitTime, timeline.lastInstant().get().getTimestamp(), + assertEquals(newCommitTime, timeline.lastInstant().get().requestedTime(), "Latest commit should be " + newCommitTime); if (filerForCommitTimeWithAssert) { // if meta cols are disabled, we can't do assertion based on assertion time assertEquals(expRecordsInThisCommit, - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, instantGenerator).count(), "Must contain " + expRecordsInThisCommit + " records"); } @@ -560,7 +572,7 @@ private JavaRDD getWriteStatusAndVerifyDeleteOperation(String newCo if (filerForCommitTimeWithAssert) { // Check that the incremental consumption from prevCommitTime assertEquals( - HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime).count(), + HoodieClientTestUtils.readCommit(basePath, sqlContext, timeline, newCommitTime, true, instantGenerator).count(), HoodieClientTestUtils.countRecordsOptionallySince(jsc, basePath, sqlContext, timeline, Option.of(prevCommitTime)), "Incremental consumption from " + prevCommitTime 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 07ec0e468676..415c8b031949 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 @@ -27,9 +27,10 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -67,6 +68,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.testutils.GenericRecordValidationTestUtils.readHFile; /** @@ -137,22 +140,18 @@ public static void overrideSparkHadoopConfiguration(SparkContext sparkContext) { private static HashMap getLatestFileIDsToFullPath(String basePath, HoodieTimeline commitTimeline, List commitsToReturn) throws IOException { HashMap fileIdToFullPath = new HashMap<>(); + TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); for (HoodieInstant commit : commitsToReturn) { HoodieCommitMetadata metadata = - HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(new StoragePath(basePath))); } return fileIdToFullPath; } public static Dataset readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, - String instantTime) { - return readCommit(basePath, sqlContext, commitTimeline, instantTime, true); - } - - public static Dataset readCommit(String basePath, SQLContext sqlContext, HoodieTimeline commitTimeline, - String instantTime, boolean filterByCommitTime) { - HoodieInstant commitInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime); + String instantTime, boolean filterByCommitTime, InstantGenerator instantGenerator) { + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime); if (!commitTimeline.containsInstant(commitInstant)) { throw new HoodieException("No commit exists at " + instantTime); } @@ -203,7 +202,7 @@ public static long countRecordsOptionallySince(JavaSparkContext jsc, String base } else if (paths[0].endsWith(HoodieFileFormat.HFILE.getFileExtension())) { Stream genericRecordStream = readHFile(HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration()), paths); if (lastCommitTimeOpt.isPresent()) { - return genericRecordStream.filter(gr -> HoodieTimeline.compareTimestamps(lastCommitTimeOpt.get(), HoodieActiveTimeline.LESSER_THAN, + return genericRecordStream.filter(gr -> compareTimestamps(lastCommitTimeOpt.get(), LESSER_THAN, gr.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString())) .count(); } else { @@ -331,7 +330,7 @@ private static Option getCommitMetadataForInstant(HoodieTa try { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); byte[] data = timeline.getInstantDetails(instant).get(); - return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class)); + return Option.of(metaClient.getCommitMetadataSerDe().deserialize(instant, data, HoodieCommitMetadata.class)); } catch (Exception e) { throw new HoodieException("Failed to read schema from commit metadata", e); } diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java index 3cb89a5c68b6..620fd8d8784a 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/HoodieSparkClientTestHarness.java @@ -106,6 +106,7 @@ import scala.Tuple2; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; @@ -702,7 +703,7 @@ HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEF HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats), Collections.EMPTY_MAP); HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata, isEmptyForAll, isEmptyCompleted); } - return new HoodieInstant(inflightOnly, "clean", instantTime); + return INSTANT_GENERATOR.createNewInstant(inflightOnly ? HoodieInstant.State.INFLIGHT : HoodieInstant.State.COMPLETED, "clean", instantTime); } protected HoodieTableMetaClient createMetaClient(SparkSession spark, String basePath) { diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java index 9e1a653f2ce9..89b372cc276c 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/testutils/SparkClientFunctionalTestHarness.java @@ -282,7 +282,7 @@ protected Stream insertRecordsToMORTable(HoodieTableMetaClient m Option deltaCommit = reloadedMetaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals(commitTime, deltaCommit.get().getTimestamp(), + assertEquals(commitTime, deltaCommit.get().requestedTime(), "Delta commit should be specified value"); Option commit = @@ -330,7 +330,7 @@ protected void updateRecordsInMORTable(HoodieTableMetaClient metaClient, List deltaCommit = reloadedMetaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals(commitTime, deltaCommit.get().getTimestamp(), + assertEquals(commitTime, deltaCommit.get().requestedTime(), "Latest Delta commit should match specified time"); Option commit = diff --git a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java index bbd3b50b50fe..de989af9c57c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/BaseHoodieTableFileIndex.java @@ -268,7 +268,7 @@ private Map> loadFileSlicesForPartitions(List latestInstant = activeTimeline.lastInstant(); try (HoodieTableFileSystemView fileSystemView = new HoodieTableFileSystemView(metaClient, activeTimeline, allFiles)) { - Option queryInstant = specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::getTimestamp)); + Option queryInstant = specifiedQueryInstant.or(() -> latestInstant.map(HoodieInstant::requestedTime)); validate(activeTimeline, queryInstant); // NOTE: For MOR table, when the compaction is inflight, we need to not only fetch the diff --git a/hudi-common/src/main/java/org/apache/hudi/common/HoodieCleanStat.java b/hudi-common/src/main/java/org/apache/hudi/common/HoodieCleanStat.java index 0913a7440f02..498d430b5c3a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/HoodieCleanStat.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/HoodieCleanStat.java @@ -186,7 +186,7 @@ public Builder withPartitionPath(String partitionPath) { public Builder withEarliestCommitRetained(Option earliestCommitToRetain) { this.earliestCommitToRetain = - (earliestCommitToRetain.isPresent()) ? earliestCommitToRetain.get().getTimestamp() : ""; + (earliestCommitToRetain.isPresent()) ? earliestCommitToRetain.get().requestedTime() : ""; return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java index 5c46b4bad0a7..4934b7a846de 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/bootstrap/index/BootstrapIndex.java @@ -30,6 +30,9 @@ import java.util.List; import java.util.Map; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * Bootstrap Index Interface. */ @@ -68,7 +71,7 @@ public BootstrapIndex(HoodieTableMetaClient metaClient) { public final boolean useIndex() { if (isPresent()) { boolean validInstantTime = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant() - .map(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), HoodieTimeline.GREATER_THAN_OR_EQUALS, + .map(i -> compareTimestamps(i.requestedTime(), GREATER_THAN_OR_EQUALS, HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS)).orElse(false); return validInstantTime && metaClient.getTableConfig().getBootstrapBasePath().isPresent(); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 1537a758bd8c..c865e41bfdd0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -131,10 +130,6 @@ public static String maskWithoutFileId(String instantTime, int taskPartitionId) .defaultValue().getFileExtension()); } - public static String getCommitFromCommitFile(String commitFileName) { - return HoodieInstant.extractTimestamp(commitFileName); - } - public static String getCommitTime(String fullFileName) { try { if (isLogFile(fullFileName)) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java index 3d077c67883d..6e2d16e025d9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/FileSlice.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.model; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; import org.apache.hudi.common.util.Option; import java.io.Serializable; @@ -150,7 +150,7 @@ public long getTotalFileSize() { */ public String getLatestInstantTime() { Option latestDeltaCommitTime = getLatestLogFile().map(HoodieLogFile::getDeltaCommitTime); - return latestDeltaCommitTime.isPresent() ? HoodieTimeline.maxInstant(latestDeltaCommitTime.get(), getBaseInstantTime()) : getBaseInstantTime(); + return latestDeltaCommitTime.isPresent() ? InstantComparison.maxInstant(latestDeltaCommitTime.get(), getBaseInstantTime()) : getBaseInstantTime(); } /** diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index 22bf37d9c3df..31710877f8e9 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -33,7 +33,6 @@ import java.io.IOException; import java.io.Serializable; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; @@ -42,9 +41,7 @@ import java.util.Map; import java.util.stream.Collectors; -import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertCommitMetadataToJsonBytes; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeCommitMetadata; -import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes; /** * All the metadata that gets stored along with a commit. @@ -496,23 +493,6 @@ public int hashCode() { return result; } - public static T fromBytes(byte[] bytes, Class clazz) throws IOException { - try { - if (bytes.length == 0) { - return clazz.newInstance(); - } - try { - return fromJsonString(fromUTF8Bytes(convertCommitMetadataToJsonBytes(deserializeCommitMetadata(bytes), org.apache.hudi.avro.model.HoodieCommitMetadata.class)), clazz); - } catch (Exception e) { - // fall back to the alternative method (0.x) - LOG.warn("Primary method failed; trying alternative deserialization method.", e); - return fromJsonString(new String(bytes, StandardCharsets.UTF_8), clazz); - } - } catch (Exception e) { - throw new IOException("unable to read commit metadata for bytes length: " + bytes.length, e); - } - } - @Override public String toString() { return "HoodieCommitMetadata{" + "partitionToWriteStats=" + partitionToWriteStats diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java index 84c42f476807..1fcc003165b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieFileGroup.java @@ -30,10 +30,10 @@ import java.util.TreeMap; import java.util.stream.Stream; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * A set of data/base files + set of log files, that make up a unit for all operations. @@ -125,7 +125,7 @@ public String getBaseInstantTime(CompletionTimeQueryView completionTimeQueryView if (completionTimeOpt.isPresent()) { for (String commitTime : fileSlices.keySet()) { // find the largest commit time that is smaller than the log delta commit completion time - if (HoodieTimeline.compareTimestamps(completionTimeOpt.get(), GREATER_THAN_OR_EQUALS, commitTime)) { + if (compareTimestamps(completionTimeOpt.get(), GREATER_THAN_OR_EQUALS, commitTime)) { return commitTime; } } @@ -150,7 +150,7 @@ public HoodieFileGroupId getFileGroupId() { * some log files, that are based off a commit or delta commit. */ private boolean isFileSliceCommitted(FileSlice slice) { - if (!compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, lastInstant.get().getTimestamp())) { + if (!compareTimestamps(slice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, lastInstant.get().requestedTime())) { return false; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java index 16fd7d2f4348..bbef386ed166 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodiePartitionMetadata.java @@ -41,13 +41,14 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.HoodieTableMetaClient.COMMIT_TIME_KEY; + /** * The metadata that goes into the meta file in each partition. */ public class HoodiePartitionMetadata { public static final String HOODIE_PARTITION_METAFILE_PREFIX = ".hoodie_partition_metadata"; - public static final String COMMIT_TIME_KEY = "commitTime"; private static final String PARTITION_DEPTH_KEY = "partitionDepth"; private static final Logger LOG = LoggerFactory.getLogger(HoodiePartitionMetadata.class); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java index 0446ace968fd..3ef962767955 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieRollingStatMetadata.java @@ -74,10 +74,6 @@ public V put(K key, V value) { } } - public static HoodieRollingStatMetadata fromBytes(byte[] bytes) throws IOException { - return HoodieCommitMetadata.fromBytes(bytes, HoodieRollingStatMetadata.class); - } - public String toJsonString() throws IOException { if (partitionToRollingStats.containsKey(null)) { LOG.info("partition path is null for " + partitionToRollingStats.get(null)); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java index f8389e3e2e1a..b934f0e811e0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableConfig.java @@ -356,13 +356,40 @@ static List> definedTableConfigs() { .collect(Collectors.toList()); } + public static HoodieTableConfig loadFromHoodieProps(HoodieStorage storage, String basePath) { + StoragePath metaPath = new StoragePath(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + return new HoodieTableConfig(storage, metaPath); + } + + private HoodieTableConfig(HoodieStorage storage, StoragePath metaPath) { + this(storage, metaPath, null, null, null, false); + } + public HoodieTableConfig(HoodieStorage storage, StoragePath metaPath, RecordMergeMode recordMergeMode, String payloadClassName, String recordMergeStrategyId) { + this(storage, metaPath, recordMergeMode, payloadClassName, recordMergeStrategyId, true); + } + + public HoodieTableConfig(HoodieStorage storage, StoragePath metaPath, RecordMergeMode recordMergeMode, String payloadClassName, + String recordMergeStrategyId, boolean autoUpdate) { super(); StoragePath propertyPath = new StoragePath(metaPath, HOODIE_PROPERTIES_FILE); LOG.info("Loading table properties from " + propertyPath); try { this.props = fetchConfigs(storage, metaPath, HOODIE_PROPERTIES_FILE, HOODIE_PROPERTIES_FILE_BACKUP, MAX_READ_RETRIES, READ_RETRY_DELAY_MSEC); + if (autoUpdate) { + autoUpdateHoodieProperties(storage, metaPath, recordMergeMode, payloadClassName, recordMergeStrategyId); + } + } catch (IOException e) { + throw new HoodieIOException("Could not load properties from " + propertyPath, e); + } + } + + private void autoUpdateHoodieProperties(HoodieStorage storage, StoragePath metaPath, + RecordMergeMode recordMergeMode, String payloadClassName, + String recordMergeStrategyId) { + StoragePath propertyPath = new StoragePath(metaPath, HOODIE_PROPERTIES_FILE); + try { boolean needStore = false; if (contains(PAYLOAD_CLASS_NAME) && payloadClassName != null && !getString(PAYLOAD_CLASS_NAME).equals(payloadClassName)) { @@ -385,7 +412,7 @@ public HoodieTableConfig(HoodieStorage storage, StoragePath metaPath, RecordMerg } } } catch (IOException e) { - throw new HoodieIOException("Could not load properties from " + propertyPath, e); + throw new HoodieIOException("Could not store properties in " + propertyPath, e); } } @@ -669,6 +696,7 @@ public HoodieTableVersion getTableInitialVersion() { public void setTableVersion(HoodieTableVersion tableVersion) { setValue(VERSION, Integer.toString(tableVersion.versionCode())); + setValue(TIMELINE_LAYOUT_VERSION, Integer.toString(tableVersion.getTimelineLayoutVersion().getVersion())); } public void setInitialVersion(HoodieTableVersion initialVersion) { 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 30b4ff73d2f4..754ecb6dc7af 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 @@ -34,13 +34,18 @@ import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieTimelineTimeZone; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.FileIOUtils; @@ -123,6 +128,8 @@ public class HoodieTableMetaClient implements Serializable { public static final String INDEX_DEFINITION_FOLDER_NAME = ".index_defs"; public static final String INDEX_DEFINITION_FILE_NAME = "index.json"; + public static final String COMMIT_TIME_KEY = "commitTime"; + // In-memory cache for archived timeline based on the start instant time // Only one entry should be present in this map private final Map archivedTimelineMap = new HashMap<>(); @@ -135,6 +142,7 @@ public class HoodieTableMetaClient implements Serializable { protected StorageConfiguration storageConf; private HoodieTableType tableType; private TimelineLayoutVersion timelineLayoutVersion; + private TimelineLayout timelineLayout; protected HoodieTableConfig tableConfig; protected HoodieActiveTimeline activeTimeline; private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); @@ -170,6 +178,7 @@ protected HoodieTableMetaClient(HoodieStorage storage, String basePath, boolean + ") than the one passed in config (" + layoutVersion.get() + ")"); } this.timelineLayoutVersion = layoutVersion.orElseGet(() -> tableConfig.getTimelineLayoutVersion().get()); + this.timelineLayout = TimelineLayout.fromVersion(timelineLayoutVersion); this.loadActiveTimelineOnLoad = loadActiveTimelineOnLoad; LOG.info("Finished Loading Table of type " + tableType + "(version=" + timelineLayoutVersion + ") from " + basePath); if (loadActiveTimelineOnLoad) { @@ -374,6 +383,10 @@ public TimelineLayoutVersion getTimelineLayoutVersion() { return timelineLayoutVersion; } + public TimelineLayout getTimelineLayout() { + return timelineLayout; + } + public Boolean isMetadataTable() { return HoodieTableMetadata.isMetadataTable(getBasePath()); } @@ -422,7 +435,7 @@ public StorageConfiguration getStorageConf() { */ public synchronized HoodieActiveTimeline getActiveTimeline() { if (activeTimeline == null) { - activeTimeline = new HoodieActiveTimeline(this); + activeTimeline = timelineLayout.getTimelineFactory().createActiveTimeline(this); } return activeTimeline; } @@ -433,7 +446,7 @@ public synchronized HoodieActiveTimeline getActiveTimeline() { * @return Active instants timeline */ public synchronized HoodieActiveTimeline reloadActiveTimeline() { - activeTimeline = new HoodieActiveTimeline(this); + activeTimeline = timelineLayout.getTimelineFactory().createActiveTimeline(this); return activeTimeline; } @@ -460,7 +473,7 @@ public String createNewInstantTime() { public String createNewInstantTime(boolean shouldLock) { TimeGenerator timeGenerator = TimeGenerators .getTimeGenerator(timeGeneratorConfig, storageConf); - return HoodieActiveTimeline.createNewInstantTime(shouldLock, timeGenerator); + return TimelineUtils.generateInstantTime(shouldLock, timeGenerator); } public HoodieTimeGeneratorConfig getTimeGeneratorConfig() { @@ -529,8 +542,8 @@ public HoodieArchivedTimeline getArchivedTimeline(String startTs, boolean useCac private HoodieArchivedTimeline instantiateArchivedTimeline(String startTs) { return StringUtils.isNullOrEmpty(startTs) - ? new HoodieArchivedTimeline(this) - : new HoodieArchivedTimeline(this, startTs); + ? timelineLayout.getTimelineFactory().createArchivedTimeline(this) + : timelineLayout.getTimelineFactory().createArchivedTimeline(this, startTs); } private static void createTableLayoutOnStorage(StorageConfiguration storageConf, @@ -696,16 +709,17 @@ public List scanHoodieInstantsFromFileSystem(Set included */ public List scanHoodieInstantsFromFileSystem(StoragePath timelinePath, Set includedExtensions, boolean applyLayoutVersionFilters) throws IOException { + final InstantGenerator instantGenerator = timelineLayout.getInstantGenerator(); Stream instantStream = HoodieTableMetaClient .scanFiles(getStorage(), timelinePath, path -> { // Include only the meta files with extensions that needs to be included - String extension = HoodieInstant.getTimelineFileExtension(path.getName()); + String extension = timelineLayout.getInstantFileNameParser().getTimelineFileExtension(path.getName()); return includedExtensions.contains(extension); - }).stream().map(HoodieInstant::new); + }).stream().map(instantGenerator::createNewInstant); if (applyLayoutVersionFilters) { - instantStream = TimelineLayout.getLayout(getTimelineLayoutVersion()).filterHoodieInstants(instantStream); + instantStream = TimelineLayout.fromVersion(getTimelineLayoutVersion()).filterHoodieInstants(instantStream); } return instantStream.sorted().collect(Collectors.toList()); } @@ -778,7 +792,7 @@ public static class Builder { private ConsistencyGuardConfig consistencyGuardConfig = ConsistencyGuardConfig.newBuilder().build(); private FileSystemRetryConfig fileSystemRetryConfig = FileSystemRetryConfig.newBuilder().build(); private HoodieMetaserverConfig metaserverConfig = HoodieMetaserverConfig.newBuilder().build(); - private Option layoutVersion = Option.of(TimelineLayoutVersion.CURR_LAYOUT_VERSION); + private Option layoutVersion = Option.empty(); public Builder setConf(StorageConfiguration conf) { this.conf = conf; @@ -867,6 +881,38 @@ public HoodieTableMetaClient build() { } } + public InstantGenerator getInstantGenerator() { + return getTimelineLayout().getInstantGenerator(); + } + + public InstantFileNameGenerator getInstantFileNameGenerator() { + return getTimelineLayout().getInstantFileNameGenerator(); + } + + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp) { + return getInstantGenerator().createNewInstant(state, action, timestamp); + } + + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime) { + return getInstantGenerator().createNewInstant(state, action, timestamp, completionTime); + } + + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime, boolean isLegacy) { + return getInstantGenerator().createNewInstant(state, action, timestamp, completionTime, isLegacy); + } + + public HoodieInstant createNewInstant(StoragePathInfo pathInfo) { + return getInstantGenerator().createNewInstant(pathInfo); + } + + public InstantFileNameParser getInstantFileNameParser() { + return getTimelineLayout().getInstantFileNameParser(); + } + + public CommitMetadataSerDe getCommitMetadataSerDe() { + return getTimelineLayout().getCommitMetadataSerDe(); + } + public static TableBuilder newTableBuilder() { return new TableBuilder(); } @@ -941,6 +987,8 @@ public TableBuilder setTableName(String tableName) { public TableBuilder setTableVersion(HoodieTableVersion tableVersion) { this.tableVersion = tableVersion; + // TimelineLayoutVersion is an internal setting which will be consistent with table version. + setTimelineLayoutVersion(tableVersion.getTimelineLayoutVersion().getVersion()); return this; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java index d6c0f93c7431..23847885e3ef 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableVersion.java @@ -18,6 +18,7 @@ package org.apache.hudi.common.table; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.exception.HoodieException; @@ -30,31 +31,38 @@ */ public enum HoodieTableVersion { // < 0.6.0 versions - ZERO(0, CollectionUtils.createImmutableList("0.3.0")), + ZERO(0, CollectionUtils.createImmutableList("0.3.0"), TimelineLayoutVersion.LAYOUT_VERSION_0), // 0.6.0 onwards - ONE(1, CollectionUtils.createImmutableList("0.6.0")), + ONE(1, CollectionUtils.createImmutableList("0.6.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.9.0 onwards - TWO(2, CollectionUtils.createImmutableList("0.9.0")), + TWO(2, CollectionUtils.createImmutableList("0.9.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.10.0 onwards - THREE(3, CollectionUtils.createImmutableList("0.10.0")), + THREE(3, CollectionUtils.createImmutableList("0.10.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.11.0 onwards - FOUR(4, CollectionUtils.createImmutableList("0.11.0")), + FOUR(4, CollectionUtils.createImmutableList("0.11.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.12.0 onwards - FIVE(5, CollectionUtils.createImmutableList("0.12.0", "0.13.0")), + FIVE(5, CollectionUtils.createImmutableList("0.12.0", "0.13.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.14.0 onwards - SIX(6, CollectionUtils.createImmutableList("0.14.0")), + SIX(6, CollectionUtils.createImmutableList("0.14.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 0.16.0 - SEVEN(7, CollectionUtils.createImmutableList("0.16.0")), + SEVEN(7, CollectionUtils.createImmutableList("0.16.0"), TimelineLayoutVersion.LAYOUT_VERSION_1), // 1.0 - EIGHT(8, CollectionUtils.createImmutableList("1.0.0")); + EIGHT(8, CollectionUtils.createImmutableList("1.0.0"), TimelineLayoutVersion.LAYOUT_VERSION_2); private final int versionCode; private final List releaseVersions; - HoodieTableVersion(int versionCode, List releaseVersions) { + private final TimelineLayoutVersion timelineLayoutVersion; + + HoodieTableVersion(int versionCode, List releaseVersions, TimelineLayoutVersion timelineLayoutVersion) { this.versionCode = versionCode; this.releaseVersions = releaseVersions; + this.timelineLayoutVersion = timelineLayoutVersion; + } + + public TimelineLayoutVersion getTimelineLayoutVersion() { + return timelineLayoutVersion; } public int versionCode() { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java index 5e3e423933c0..19e7f67e1b41 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/TableSchemaResolver.java @@ -300,8 +300,8 @@ public Schema readSchemaFromLastCompaction(Option lastCompactionC "Could not read schema from last compaction, no compaction commits found on path " + metaClient)); // Read from the compacted file wrote - HoodieCommitMetadata compactionMetadata = HoodieCommitMetadata - .fromBytes(activeTimeline.getInstantDetails(lastCompactionCommit).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata compactionMetadata = metaClient.getCommitMetadataSerDe().deserialize( + lastCompactionCommit, activeTimeline.getInstantDetails(lastCompactionCommit).get(),HoodieCommitMetadata.class); String filePath = compactionMetadata.getFileIdAndFullPaths(metaClient.getBasePath()).values().stream().findAny() .orElseThrow(() -> new IllegalArgumentException("Could not find any data file written for compaction " + lastCompactionCommit + ", could not get schema for table " + metaClient.getBasePath())); @@ -345,8 +345,8 @@ public Option getTableInternalSchemaFromCommitMetadata() { HoodieTimeline timeline = completedInstants .filter(instant -> { // consider only instants that can update/change schema. try { - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(completedInstants.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + instant, completedInstants.getInstantDetails(instant).get(), HoodieCommitMetadata.class); return WriteOperationType.canUpdateSchema(commitMetadata.getOperationType()); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); @@ -459,7 +459,7 @@ private HoodieCommitMetadata getCachedCommitMetadata(HoodieInstant instant) { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); byte[] data = timeline.getInstantDetails(missingInstant).get(); try { - return HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + return metaClient.getCommitMetadataSerDe().deserialize(missingInstant, data, HoodieCommitMetadata.class); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", missingInstant), e); } 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 3b15373044aa..77ad0ed3fab3 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 @@ -145,7 +145,7 @@ public Map> extractCDCFileSplits() { Option latestFileSliceOpt = getOrCreateFsView().fetchLatestFileSlice(partition, fileId); if (latestFileSliceOpt.isPresent()) { HoodieFileGroupId fileGroupId = new HoodieFileGroupId(partition, fileId); - HoodieCDCFileSplit changeFile = new HoodieCDCFileSplit(instant.getTimestamp(), + HoodieCDCFileSplit changeFile = new HoodieCDCFileSplit(instant.requestedTime(), REPLACE_COMMIT, new ArrayList<>(), latestFileSliceOpt, Option.empty()); if (!fgToCommitChanges.containsKey(fileGroupId)) { fgToCommitChanges.put(fileGroupId, new ArrayList<>()); @@ -220,7 +220,7 @@ private void initInstantAndCommitMetadata() { this.commits = activeTimeLine.getInstantsAsStream() .filter(instant -> instant.isCompleted() - && instantRange.isInRange(instant.getTimestamp()) + && instantRange.isInRange(instant.requestedTime()) && requiredActions.contains(instant.getAction().toLowerCase(Locale.ROOT)) ).map(instant -> { final HoodieCommitMetadata commitMetadata; @@ -250,7 +250,7 @@ private HoodieCDCFileSplit parseWriteStat( WriteOperationType operation) { final StoragePath basePath = metaClient.getBasePath(); final HoodieStorage storage = metaClient.getStorage(); - final String instantTs = instant.getTimestamp(); + final String instantTs = instant.requestedTime(); HoodieCDCFileSplit cdcFileSplit; if (CollectionUtils.isNullOrEmpty(writeStat.getCdcStats())) { @@ -294,7 +294,7 @@ private HoodieCDCFileSplit parseWriteStat( new HoodieIOException("Can not get the previous version of the base file") ); FileSlice beforeFileSlice = null; - FileSlice currentFileSlice = new FileSlice(fileGroupId, instant.getTimestamp(), + FileSlice currentFileSlice = new FileSlice(fileGroupId, instant.requestedTime(), new HoodieBaseFile( storage.getPathInfo(new StoragePath(basePath, writeStat.getPath()))), new ArrayList<>()); @@ -337,7 +337,7 @@ private Option getDependentFileSliceForLogFile( .collect(Collectors.toList()); List logFiles = storage.listDirectEntries(logFilePaths).stream() .map(HoodieLogFile::new).collect(Collectors.toList()); - return Option.of(new FileSlice(fgId, instant.getTimestamp(), baseFile, logFiles)); + return Option.of(new FileSlice(fgId, instant.requestedTime(), baseFile, logFiles)); } catch (Exception e) { throw new HoodieException("Fail to get the dependent file slice for a log file", e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java index 01353175912b..76a6c7cf1d49 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/AbstractHoodieLogRecordReader.java @@ -31,7 +31,6 @@ import org.apache.hudi.common.table.log.block.HoodieDataBlock; import org.apache.hudi.common.table.log.block.HoodieDeleteBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ClosableIterator; @@ -71,6 +70,8 @@ import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -253,7 +254,7 @@ private void scanInternalV1(Option keySpecOpt) { final String instantTime = logBlock.getLogBlockHeader().get(INSTANT_TIME); totalLogBlocks.incrementAndGet(); if (logBlock.isDataOrDeleteBlock()) { - if (HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), HoodieTimeline.GREATER_THAN, this.latestInstantTime)) { + if (compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), GREATER_THAN, this.latestInstantTime)) { // Skip processing a data or delete block with the instant time greater than the latest instant time used by this log record reader continue; } @@ -439,7 +440,7 @@ private void scanInternalV2(Option keySpecOption, boolean skipProcessin continue; } if (logBlock.isDataOrDeleteBlock() - && HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), HoodieTimeline.GREATER_THAN, this.latestInstantTime)) { + && compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), GREATER_THAN, this.latestInstantTime)) { // Skip processing a data or delete block with the instant time greater than the latest instant time used by this log record reader continue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java index 2279b2f50598..99ac917f85c8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/BaseHoodieLogRecordReader.java @@ -66,6 +66,8 @@ import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType.TARGET_INSTANT_TIME; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.COMMAND_BLOCK; import static org.apache.hudi.common.table.log.block.HoodieLogBlock.HoodieLogBlockType.CORRUPT_BLOCK; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ValidationUtils.checkState; /** @@ -250,7 +252,7 @@ private void scanInternalV1(Option keySpecOpt) { } totalLogBlocks.incrementAndGet(); if (logBlock.isDataOrDeleteBlock()) { - if (HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), HoodieTimeline.GREATER_THAN, this.latestInstantTime)) { + if (compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), GREATER_THAN, this.latestInstantTime)) { // Skip processing a data or delete block with the instant time greater than the latest instant time used by this log record reader continue; } @@ -581,7 +583,7 @@ private void scanInternalV2(Option keySpecOption, boolean skipProcessin continue; } if (logBlock.isDataOrDeleteBlock() - && HoodieTimeline.compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), HoodieTimeline.GREATER_THAN, this.latestInstantTime)) { + && compareTimestamps(logBlock.getLogBlockHeader().get(INSTANT_TIME), GREATER_THAN, this.latestInstantTime)) { // Skip processing a data or delete block with the instant time greater than the latest instant time used by this log record reader continue; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/log/InstantRange.java b/hudi-common/src/main/java/org/apache/hudi/common/table/log/InstantRange.java index 6dfc09dbf4f1..6d6d29bb40f6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/log/InstantRange.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/log/InstantRange.java @@ -18,7 +18,6 @@ package org.apache.hudi.common.table.log; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -30,6 +29,11 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * An instant range used for incremental reader filtering. */ @@ -96,10 +100,10 @@ public OpenClosedRange(String startInstant, String endInstant) { @Override public boolean isInRange(String instant) { - boolean validAgainstStart = HoodieTimeline.compareTimestamps(instant, HoodieTimeline.GREATER_THAN, startInstant.get()); + boolean validAgainstStart = compareTimestamps(instant, GREATER_THAN, startInstant.get()); // if there is an end instant, check against it, otherwise assume +INF and its always valid. boolean validAgainstEnd = endInstant - .map(e -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, e)) + .map(e -> compareTimestamps(instant, LESSER_THAN_OR_EQUALS, e)) .orElse(true); return validAgainstStart && validAgainstEnd; } @@ -116,10 +120,10 @@ public OpenClosedRangeNullableBoundary(String startInstant, String endInstant) { @Override public boolean isInRange(String instant) { boolean validAgainstStart = startInstant - .map(s -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.GREATER_THAN, s)) + .map(s -> compareTimestamps(instant, GREATER_THAN, s)) .orElse(true); boolean validAgainstEnd = endInstant - .map(e -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, e)) + .map(e -> compareTimestamps(instant, LESSER_THAN_OR_EQUALS, e)) .orElse(true); return validAgainstStart && validAgainstEnd; @@ -134,9 +138,9 @@ public ClosedClosedRange(String startInstant, String endInstant) { @Override public boolean isInRange(String instant) { - boolean validAgainstStart = HoodieTimeline.compareTimestamps(instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, startInstant.get()); + boolean validAgainstStart = compareTimestamps(instant, GREATER_THAN_OR_EQUALS, startInstant.get()); boolean validAgainstEnd = endInstant - .map(e -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, e)) + .map(e -> compareTimestamps(instant, LESSER_THAN_OR_EQUALS, e)) .orElse(true); return validAgainstStart && validAgainstEnd; } @@ -153,10 +157,10 @@ public ClosedClosedRangeNullableBoundary(String startInstant, String endInstant) @Override public boolean isInRange(String instant) { boolean validAgainstStart = startInstant - .map(s -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, s)) + .map(s -> compareTimestamps(instant, GREATER_THAN_OR_EQUALS, s)) .orElse(true); boolean validAgainstEnd = endInstant - .map(e -> HoodieTimeline.compareTimestamps(instant, HoodieTimeline.LESSER_THAN_OR_EQUALS, e)) + .map(e -> compareTimestamps(instant, LESSER_THAN_OR_EQUALS, e)) .orElse(true); return validAgainstStart && validAgainstEnd; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java b/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java index 978dfb2026b7..9f22586a560d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/read/IncrementalQueryAnalyzer.java @@ -30,6 +30,10 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.VisibleForTesting; import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; import javax.annotation.Nullable; @@ -106,6 +110,7 @@ */ public class IncrementalQueryAnalyzer { public static final String START_COMMIT_EARLIEST = "earliest"; + private static final Logger LOG = LoggerFactory.getLogger(IncrementalQueryAnalyzer.class); private final HoodieTableMetaClient metaClient; private final Option startCompletionTime; @@ -155,7 +160,7 @@ public static Builder builder() { * @return An incremental query context including the instant time range info. */ public QueryContext analyze() { - try (CompletionTimeQueryView completionTimeQueryView = new CompletionTimeQueryView(this.metaClient)) { + try (CompletionTimeQueryView completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(this.metaClient)) { if (completionTimeQueryView.isEmptyTable()) { // no dataset committed in the table return QueryContext.EMPTY; @@ -175,7 +180,7 @@ public QueryContext analyze() { List activeInstants = new ArrayList<>(); HoodieTimeline archivedReadTimeline = null; if (!activeInstantTime.isEmpty()) { - activeInstants = filteredTimeline.getInstantsAsStream().filter(instant -> instantTimeSet.contains(instant.getTimestamp())).collect(Collectors.toList()); + activeInstants = filteredTimeline.getInstantsAsStream().filter(instant -> instantTimeSet.contains(instant.requestedTime())).collect(Collectors.toList()); if (limit > 0 && limit < activeInstants.size()) { // streaming read speed limit, limits the maximum number of commits allowed to read for each run activeInstants = activeInstants.subList(0, limit); @@ -183,9 +188,9 @@ public QueryContext analyze() { } if (!archivedInstantTime.isEmpty()) { archivedReadTimeline = getArchivedReadTimeline(metaClient, archivedInstantTime.get(0)); - archivedInstants = archivedReadTimeline.getInstantsAsStream().filter(instant -> instantTimeSet.contains(instant.getTimestamp())).collect(Collectors.toList()); + archivedInstants = archivedReadTimeline.getInstantsAsStream().filter(instant -> instantTimeSet.contains(instant.requestedTime())).collect(Collectors.toList()); } - List instants = Stream.concat(archivedInstants.stream(), activeInstants.stream()).map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List instants = Stream.concat(archivedInstants.stream(), activeInstants.stream()).map(HoodieInstant::requestedTime).collect(Collectors.toList()); if (instants.isEmpty()) { // no instants completed within the give time range, returns early. return QueryContext.EMPTY; @@ -200,6 +205,9 @@ public QueryContext analyze() { startCompletionTime.isEmpty() ? lastInstant : instants.get(0); String endInstant = endCompletionTime.isEmpty() ? null : lastInstant; return QueryContext.create(startInstant, endInstant, instants, archivedInstants, activeInstants, filteredTimeline, archivedReadTimeline); + } catch (Exception ex) { + LOG.error("Got exception when generating incremental query info", ex); + throw new HoodieException(ex); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveAction.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveAction.java index fe07a7ff6b01..74ef6c0d1dbf 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveAction.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveAction.java @@ -99,7 +99,7 @@ public String getPendingAction() { } public String getInstantTime() { - return getCompleted().getTimestamp(); + return getCompleted().requestedTime(); } public String getCompletionTime() { @@ -174,7 +174,7 @@ protected Option getPendingInstant() { @Override public int compareTo(ActiveAction other) { - return this.getCompleted().getTimestamp().compareTo(other.getCompleted().getTimestamp()); + return this.getCompleted().requestedTime().compareTo(other.getCompleted().requestedTime()); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveTimelineUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveTimelineUtils.java new file mode 100644 index 000000000000..06aaea75d8c6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ActiveTimelineUtils.java @@ -0,0 +1,23 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +public class ActiveTimelineUtils { + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ArchivedTimelineLoader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ArchivedTimelineLoader.java new file mode 100644 index 000000000000..96a765e4085d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/ArchivedTimelineLoader.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.avro.generic.GenericRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; + +import javax.annotation.Nullable; +import java.io.Serializable; +import java.util.function.BiConsumer; +import java.util.function.Function; + +public interface ArchivedTimelineLoader extends Serializable { + + /** + * Loads the instants from the timeline. + * + * @param metaClient The meta client. + * @param filter The time range filter where the target instant belongs to. + * @param loadMode The load mode. + * @param commitsFilter Filter of the instant type. + * @param recordConsumer Consumer of the instant record payload. + */ + public void loadInstants( + HoodieTableMetaClient metaClient, + @Nullable HoodieArchivedTimeline.TimeRangeFilter filter, + HoodieArchivedTimeline.LoadMode loadMode, + Function commitsFilter, + BiConsumer recordConsumer); +} 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/BaseHoodieTimeline.java similarity index 58% rename from hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java rename to hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java index c2f570fd10dd..18efaccfe921 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/BaseHoodieTimeline.java @@ -19,7 +19,6 @@ package org.apache.hudi.common.table.timeline; import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -42,9 +41,11 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.common.table.timeline.HoodieInstant.COMPLETION_TIME_COMPARATOR; -import static org.apache.hudi.common.table.timeline.HoodieInstant.INSTANT_TIME_COMPARATOR; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; /** @@ -53,9 +54,9 @@ * * @see HoodieTimeline */ -public class HoodieDefaultTimeline implements HoodieTimeline { +public abstract class BaseHoodieTimeline implements HoodieTimeline { - private static final Logger LOG = LoggerFactory.getLogger(HoodieDefaultTimeline.class); + private static final Logger LOG = LoggerFactory.getLogger(BaseHoodieTimeline.class); private static final long serialVersionUID = 1L; @@ -66,25 +67,40 @@ public class HoodieDefaultTimeline implements HoodieTimeline { // for efficient #contains queries. private transient volatile Set instantTimeSet; // for efficient #isPendingClusteringInstant queries - private transient volatile Set pendingClusteringInstants; + protected transient volatile Set pendingClusteringInstants; // for efficient #isBeforeTimelineStarts check. private transient volatile Option firstNonSavepointCommit; // for efficient #isBeforeTimelineStartsByCompletionTime private transient volatile Option firstNonSavepointCommitByCompletionTime; private String timelineHash; - public HoodieDefaultTimeline(Stream instants, Function> details) { + protected TimelineFactory factory; + protected InstantComparator instantComparator; + protected InstantGenerator instantGenerator; + + public BaseHoodieTimeline(TimelineLayout layout) { + this.factory = layout.getTimelineFactory(); + this.instantComparator = layout.getInstantComparator(); + this.instantGenerator = layout.getInstantGenerator(); + } + + public BaseHoodieTimeline(Stream instants, Function> details, + TimelineFactory factory, InstantComparator instantComparator, InstantGenerator instantGenerator) { this.details = details; + this.factory = factory; + this.instantComparator = instantComparator; + this.instantGenerator = instantGenerator; setInstants(instants.collect(Collectors.toList())); } + @Override public void setInstants(List instants) { this.instants = instants; this.timelineHash = computeTimelineHash(this.instants); clearState(); } - public void appendInstants(List newInstants) { + protected void appendInstants(List newInstants) { if (newInstants.isEmpty()) { // the new instants is empty, nothing to do. return; @@ -99,72 +115,64 @@ public void appendInstants(List newInstants) { clearState(); } - /** - * For serializing and de-serializing. - * - * @deprecated - */ - public HoodieDefaultTimeline() { - } - @Override public HoodieTimeline filterInflights() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isInflight), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isInflight), details); } @Override public HoodieTimeline filterInflightsAndRequested() { - return new HoodieDefaultTimeline( + return factory.createDefaultTimeline( getInstantsAsStream().filter(i -> i.getState().equals(State.REQUESTED) || i.getState().equals(State.INFLIGHT)), details); } @Override public HoodieTimeline filterPendingExcludingCompaction() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) - && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) + && (!instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); } @Override public HoodieTimeline filterPendingExcludingLogCompaction() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) + return factory.createDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) && (!instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION))), details); } @Override public HoodieTimeline filterPendingExcludingCompactionAndLogCompaction() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(instant -> (!instant.isCompleted()) + return factory.createDefaultTimeline(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(getInstantsAsStream().filter(HoodieInstant::isCompleted), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(HoodieInstant::isCompleted), details); } @Override public HoodieTimeline filterCompletedAndCompactionInstants() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() - || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() + || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION)), details); } @Override public HoodieTimeline filterCompletedOrMajorOrMinorCompactionInstants() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)), details); } @Override - public HoodieDefaultTimeline filterCompletedInstantsOrRewriteTimeline() { + public HoodieTimeline filterCompletedInstantsOrRewriteTimeline() { Set validActions = CollectionUtils.createSet(COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> s.isCompleted() || validActions.contains(s.getAction())), details); } @Override - public HoodieDefaultTimeline getWriteTimeline() { + public HoodieTimeline getWriteTimeline() { Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION); - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); } @Override @@ -172,65 +180,53 @@ public HoodieTimeline getContiguousCompletedWriteTimeline() { Option earliestPending = getWriteTimeline().filterInflightsAndRequested().firstInstant(); if (earliestPending.isPresent()) { return getWriteTimeline().filterCompletedInstants() - .filter(instant -> compareTimestamps(instant.getTimestamp(), LESSER_THAN, earliestPending.get().getTimestamp())); + .filter(instant -> compareTimestamps(instant.requestedTime(), LESSER_THAN, earliestPending.get().requestedTime())); } return getWriteTimeline().filterCompletedInstants(); } @Override public HoodieTimeline getCompletedReplaceTimeline() { - return new HoodieDefaultTimeline( + return factory.createDefaultTimeline( getInstantsAsStream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(HoodieInstant::isCompleted), details); } @Override public HoodieTimeline filterPendingReplaceTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( + return factory.createDefaultTimeline(getInstantsAsStream().filter( s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()), details); } @Override - public HoodieTimeline filterPendingClusteringTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( - s -> s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) && !s.isCompleted()), details); - } + public abstract HoodieTimeline filterPendingClusteringTimeline(); @Override - public HoodieTimeline filterPendingReplaceOrClusteringTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( - s -> (s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) || s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) - && !s.isCompleted()), details); - } + public abstract HoodieTimeline filterPendingReplaceOrClusteringTimeline(); @Override - public HoodieTimeline filterPendingReplaceClusteringAndCompactionTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( - s -> !s.isCompleted() && (s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) - || s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) - || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); - } + public abstract HoodieTimeline filterPendingReplaceClusteringAndCompactionTimeline(); @Override public HoodieTimeline filterPendingRollbackTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( + return factory.createDefaultTimeline(getInstantsAsStream().filter( s -> s.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterRequestedRollbackTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter( + return factory.createDefaultTimeline(getInstantsAsStream().filter( s -> s.getAction().equals(HoodieTimeline.ROLLBACK_ACTION) && s.isRequested()), details); } @Override public HoodieTimeline filterPendingCompactionTimeline() { - return new HoodieDefaultTimeline( + return factory.createDefaultTimeline( getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterPendingLogCompactionTimeline() { - return new HoodieDefaultTimeline( + return factory.createDefaultTimeline( getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); } @@ -239,196 +235,164 @@ public HoodieTimeline filterPendingLogCompactionTimeline() { */ @Override public HoodieTimeline filterPendingMajorOrMinorCompactionTimeline() { - return new HoodieDefaultTimeline( + return factory.createDefaultTimeline( getInstantsAsStream().filter(s -> s.getAction().equals(HoodieTimeline.COMPACTION_ACTION) || s.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION) && !s.isCompleted()), details); } @Override - public HoodieDefaultTimeline findInstantsInRange(String startTs, String endTs) { - return new HoodieDefaultTimeline( - getInstantsAsStream().filter(s -> HoodieTimeline.isInRange(s.getTimestamp(), startTs, endTs)), details); + public HoodieTimeline findInstantsInRange(String startTs, String endTs) { + return factory.createDefaultTimeline( + getInstantsAsStream().filter(s -> InstantComparison.isInRange(s.requestedTime(), startTs, endTs)), details); } @Override - public HoodieDefaultTimeline findInstantsInClosedRange(String startTs, String endTs) { - return new HoodieDefaultTimeline( - instants.stream().filter(instant -> HoodieTimeline.isInClosedRange(instant.getTimestamp(), startTs, endTs)), details); + public HoodieTimeline findInstantsInClosedRange(String startTs, String endTs) { + return factory.createDefaultTimeline( + instants.stream().filter(instant -> InstantComparison.isInClosedRange(instant.requestedTime(), startTs, endTs)), details); } @Override - public HoodieDefaultTimeline findInstantsInRangeByCompletionTime(String startTs, String endTs) { - return new HoodieDefaultTimeline( - getInstantsAsStream().filter(s -> s.getCompletionTime() != null && HoodieTimeline.isInClosedRange(s.getCompletionTime(), startTs, endTs)), + public HoodieTimeline findInstantsInRangeByCompletionTime(String startTs, String endTs) { + return factory.createDefaultTimeline( + getInstantsAsStream().filter(s -> s.getCompletionTime() != null && InstantComparison.isInClosedRange(s.getCompletionTime(), startTs, endTs)), details); } @Override - public HoodieDefaultTimeline findInstantsModifiedAfterByCompletionTime(String instantTime) { - return new HoodieDefaultTimeline(instants.stream() + public HoodieTimeline findInstantsModifiedAfterByCompletionTime(String instantTime) { + return factory.createDefaultTimeline(instants.stream() // either pending or completionTime greater than instantTime - .filter(s -> (s.getCompletionTime() == null && compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)) - || (s.getCompletionTime() != null && compareTimestamps(s.getCompletionTime(), GREATER_THAN, instantTime) && !s.getTimestamp().equals(instantTime))), + .filter(s -> (s.getCompletionTime() == null && compareTimestamps(s.requestedTime(), GREATER_THAN, instantTime)) + || (s.getCompletionTime() != null && compareTimestamps(s.getCompletionTime(), GREATER_THAN, instantTime) && !s.requestedTime().equals(instantTime))), details); } @Override - public HoodieDefaultTimeline findInstantsAfter(String instantTime, int numCommits) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)).limit(numCommits), + public HoodieTimeline findInstantsAfter(String instantTime, int numCommits) { + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), GREATER_THAN, instantTime)).limit(numCommits), details); } @Override public HoodieTimeline findInstantsAfter(String instantTime) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN, instantTime)), details); + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), GREATER_THAN, instantTime)), details); } @Override - public HoodieDefaultTimeline findInstantsAfterOrEquals(String commitTime, int numCommits) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, commitTime)) + public HoodieTimeline findInstantsAfterOrEquals(String commitTime, int numCommits) { + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), GREATER_THAN_OR_EQUALS, commitTime)) .limit(numCommits), details); } @Override public HoodieTimeline findInstantsAfterOrEquals(String commitTime) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), GREATER_THAN_OR_EQUALS, commitTime)), details); + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), GREATER_THAN_OR_EQUALS, commitTime)), details); } @Override - public HoodieDefaultTimeline findInstantsBefore(String instantTime) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN, instantTime)), - details); + public HoodieTimeline findInstantsBefore(String instantTime) { + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), LESSER_THAN, instantTime)), + details); } @Override public Option findInstantBefore(String instantTime) { return Option.fromJavaOptional(instants.stream() - .filter(instant -> compareTimestamps(instant.getTimestamp(), LESSER_THAN, instantTime)) - .max(Comparator.comparing(HoodieInstant::getTimestamp))); + .filter(instant -> compareTimestamps(instant.requestedTime(), LESSER_THAN, instantTime)) + .max(Comparator.comparing(HoodieInstant::requestedTime))); } @Override - public HoodieDefaultTimeline findInstantsBeforeOrEquals(String instantTime) { - return new HoodieDefaultTimeline(getInstantsAsStream() - .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN_OR_EQUALS, instantTime)), + public HoodieTimeline findInstantsBeforeOrEquals(String instantTime) { + return factory.createDefaultTimeline(getInstantsAsStream() + .filter(s -> compareTimestamps(s.requestedTime(), LESSER_THAN_OR_EQUALS, instantTime)), details); } @Override public HoodieTimeline filter(Predicate filter) { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(filter), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(filter), details); } @Override public HoodieTimeline filterPendingIndexTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && !s.isCompleted()), details); } @Override public HoodieTimeline filterCompletedIndexTimeline() { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> s.getAction().equals(INDEXING_ACTION) && s.isCompleted()), details); } - /** - * Get all instants (commits, delta commits) that produce new data, in the active timeline. - */ - public HoodieTimeline getCommitsTimeline() { - return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION)); - } - - /** - * Get all instants (commits, delta commits, replace, compaction) that produce new data or merge file, in the active timeline. - */ + @Override public HoodieTimeline getCommitsAndCompactionTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION, COMPACTION_ACTION)); } - /** - * Get all instants (commits, delta commits, compaction, clean, savepoint, rollback, replace commits, index) that result in actions, - * in the active timeline. - */ + @Override public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION, INDEXING_ACTION, LOG_COMPACTION_ACTION)); } - /** - * Get only pure commit and replace commits (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getCommitAndReplaceTimeline() { //TODO: Make sure this change does not break existing functionality. return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION)); } - /** - * Get only pure commits (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getCommitTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION)); } - /** - * Get only the delta commits (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getDeltaCommitTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(DELTA_COMMIT_ACTION), - (Function> & Serializable) this::getInstantDetails); + return factory.createDefaultTimeline(filterInstantsByAction(DELTA_COMMIT_ACTION), + (Function> & Serializable) this::getInstantDetails); } - /** - * Get a timeline of a specific set of actions. useful to create a merged timeline of multiple actions. - * - * @param actions actions allowed in the timeline - */ + @Override public HoodieTimeline getTimelineOfActions(Set actions) { - return new HoodieDefaultTimeline(getInstantsAsStream().filter(s -> actions.contains(s.getAction())), - (Function> & Serializable) this::getInstantDetails); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> actions.contains(s.getAction())), + (Function> & Serializable) this::getInstantDetails); } - /** - * Get only the cleaner action (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getCleanerTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(CLEAN_ACTION), - (Function> & Serializable) this::getInstantDetails); + return factory.createDefaultTimeline(filterInstantsByAction(CLEAN_ACTION), + (Function> & Serializable) this::getInstantDetails); } - /** - * Get only the rollback action (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getRollbackTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION), + return factory.createDefaultTimeline(filterInstantsByAction(ROLLBACK_ACTION), (Function> & Serializable) this::getInstantDetails); } - /** - * Get only the rollback and restore action (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getRollbackAndRestoreTimeline() { return getTimelineOfActions(CollectionUtils.createSet(ROLLBACK_ACTION, RESTORE_ACTION)); } - /** - * Get only the save point action (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getSavePointTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION), - (Function> & Serializable) this::getInstantDetails); + return factory.createDefaultTimeline(filterInstantsByAction(SAVEPOINT_ACTION), + (Function> & Serializable) this::getInstantDetails); } - /** - * Get only the restore action (inflight and completed) in the active timeline. - */ + @Override public HoodieTimeline getRestoreTimeline() { - return new HoodieDefaultTimeline(filterInstantsByAction(RESTORE_ACTION), - (Function> & Serializable) this::getInstantDetails); + return factory.createDefaultTimeline(filterInstantsByAction(RESTORE_ACTION), + (Function> & Serializable) this::getInstantDetails); } protected Stream filterInstantsByAction(String action) { @@ -520,27 +484,27 @@ public List getInstants() { @Override public Stream getReverseOrderedInstants() { - return getInstantsAsStream().sorted(INSTANT_TIME_COMPARATOR.reversed()); + return getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()); } @Override public Option getLatestCompletionTime() { return Option.fromJavaOptional(getInstantsAsStream().filter(s -> s.getCompletionTime() != null) - .max(HoodieInstant.COMPLETION_TIME_COMPARATOR) + .max(instantComparator.completionTimeOrderedComparator()) .map(HoodieInstant::getCompletionTime)); } @Override public Stream getInstantsOrderedByCompletionTime() { return getInstantsAsStream().filter(s -> s.getCompletionTime() != null) - .sorted(HoodieInstant.COMPLETION_TIME_COMPARATOR); + .sorted(instantComparator.completionTimeOrderedComparator()); } @Override public boolean isBeforeTimelineStarts(String instant) { Option firstNonSavepointCommit = getFirstNonSavepointCommit(); return firstNonSavepointCommit.isPresent() - && compareTimestamps(instant, LESSER_THAN, firstNonSavepointCommit.get().getTimestamp()); + && compareTimestamps(instant, LESSER_THAN, firstNonSavepointCommit.get().requestedTime()); } @Override @@ -556,7 +520,7 @@ public Option getFirstNonSavepointCommit() { synchronized (this) { if (this.firstNonSavepointCommit == null) { this.firstNonSavepointCommit = - findFirstNonSavepointCommit(this.instants, INSTANT_TIME_COMPARATOR); + findFirstNonSavepointCommit(this.instants, instantComparator.requestedTimeOrderedComparator()); } } } @@ -571,7 +535,7 @@ public Option getFirstNonSavepointCommitByCompletionTime() { this.firstNonSavepointCommitByCompletionTime = findFirstNonSavepointCommit( this.instants.stream().filter(HoodieInstant::isCompleted).collect(Collectors.toList()), - COMPLETION_TIME_COMPARATOR); + instantComparator.completionTimeOrderedComparator()); } } } @@ -579,40 +543,16 @@ public Option getFirstNonSavepointCommitByCompletionTime() { } @Override - public Option getLastClusteringInstant() { - return Option.fromJavaOptional(getCommitsTimeline().filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) - || s.getAction().equalsIgnoreCase(HoodieTimeline.CLUSTERING_ACTION)) - .getReverseOrderedInstants() - .filter(i -> ClusteringUtils.isClusteringInstant(this, i)) - .findFirst()); - } + public abstract Option getLastClusteringInstant(); @Override - public Option getFirstPendingClusterInstant() { - return getLastOrFirstPendingClusterInstant(false); - } + public abstract Option getFirstPendingClusterInstant(); @Override - public Option getLastPendingClusterInstant() { - return getLastOrFirstPendingClusterInstant(true); - } - - private Option getLastOrFirstPendingClusterInstant(boolean isLast) { - HoodieTimeline pendingClusteringTimeline = filterPendingReplaceOrClusteringTimeline(); - Stream clusterStream; - if (isLast) { - clusterStream = pendingClusteringTimeline.getReverseOrderedInstants(); - } else { - clusterStream = pendingClusteringTimeline.getInstantsAsStream(); - } - return Option.fromJavaOptional(clusterStream - .filter(i -> ClusteringUtils.isClusteringInstant(this, i)).findFirst()); - } + public abstract Option getLastPendingClusterInstant(); @Override - public boolean isPendingClusteringInstant(String instantTime) { - return getOrCreatePendingClusteringInstantSet().contains(instantTime); - } + public abstract boolean isPendingClusteringInstant(String instantTime); @Override public Option getInstantDetails(HoodieInstant instant) { @@ -633,34 +573,13 @@ private Set getOrCreateInstantSet() { if (this.instantTimeSet == null) { synchronized (this) { if (this.instantTimeSet == null) { - this.instantTimeSet = this.instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + this.instantTimeSet = this.instants.stream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); } } } return this.instantTimeSet; } - private Set getOrCreatePendingClusteringInstantSet() { - if (this.pendingClusteringInstants == null) { - synchronized (this) { - if (this.pendingClusteringInstants == null) { - List pendingClusterInstants = getCommitsTimeline().filterPendingReplaceOrClusteringTimeline().getInstants(); - // Validate that there are no instants with same timestamp - pendingClusterInstants.stream().collect(Collectors.groupingBy(HoodieInstant::getTimestamp)).forEach((timestamp, instants) -> { - if (instants.size() > 1) { - throw new IllegalStateException("Multiple instants with same timestamp: " + timestamp + " instants: " + instants); - } - }); - // Filter replace commits down to those that are due to clustering - this.pendingClusteringInstants = pendingClusterInstants.stream() - .filter(instant -> ClusteringUtils.isClusteringInstant(this, instant)) - .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - } - } - } - return this.pendingClusteringInstants; - } - /** * Returns the first non savepoint commit on the timeline. * @@ -673,12 +592,12 @@ private static Option findFirstNonSavepointCommit( Comparator instantComparator) { Set savepointTimestamps = instants.stream() .filter(entry -> entry.getAction().equals(HoodieTimeline.SAVEPOINT_ACTION)) - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .collect(Collectors.toSet()); // 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. return Option.fromJavaOptional(instants.stream() - .filter(entry -> !savepointTimestamps.contains(entry.getTimestamp())) + .filter(entry -> !savepointTimestamps.contains(entry.requestedTime())) .min(instantComparator)); } @@ -687,10 +606,8 @@ private void clearState() { firstNonSavepointCommit = null; } - /** - * Merge this timeline with the given timeline. - */ - public HoodieDefaultTimeline mergeTimeline(HoodieDefaultTimeline timeline) { + @Override + public HoodieTimeline mergeTimeline(HoodieTimeline timeline) { Stream instantStream = Stream.concat(getInstantsAsStream(), timeline.getInstantsAsStream()).sorted(); Function> details = instant -> { if (getInstantsAsStream().anyMatch(i -> i.equals(instant))) { @@ -699,7 +616,7 @@ public HoodieDefaultTimeline mergeTimeline(HoodieDefaultTimeline timeline) { return timeline.getInstantDetails(instant); } }; - return new HoodieDefaultTimeline(instantStream, details); + return factory.createDefaultTimeline(instantStream, details); } /** @@ -710,7 +627,7 @@ private String computeTimelineHash(List instants) { try { md = MessageDigest.getInstance(HASHING_ALGORITHM); instants.forEach(i -> md - .update(getUTF8Bytes(StringUtils.joinUsingDelim("_", i.getTimestamp(), i.getAction(), i.getState().name())))); + .update(getUTF8Bytes(StringUtils.joinUsingDelim("_", i.requestedTime(), i.getAction(), i.getState().name())))); } catch (NoSuchAlgorithmException nse) { throw new HoodieException(nse); } @@ -725,10 +642,10 @@ private static List mergeInstants(List instants1, // some optimizations are based on the assumption all the instant lists are already sorted. // skip when one list contains all the instants of the other one. List merged; - if (HoodieTimeline.compareTimestamps(instants1.get(instants1.size() - 1).getTimestamp(), LESSER_THAN_OR_EQUALS, instants2.get(0).getTimestamp())) { + if (InstantComparison.compareTimestamps(instants1.get(instants1.size() - 1).requestedTime(), LESSER_THAN_OR_EQUALS, instants2.get(0).requestedTime())) { merged = new ArrayList<>(instants1); merged.addAll(instants2); - } else if (HoodieTimeline.compareTimestamps(instants2.get(instants2.size() - 1).getTimestamp(), LESSER_THAN_OR_EQUALS, instants1.get(0).getTimestamp())) { + } else if (InstantComparison.compareTimestamps(instants2.get(instants2.size() - 1).requestedTime(), LESSER_THAN_OR_EQUALS, instants1.get(0).requestedTime())) { merged = new ArrayList<>(instants2); merged.addAll(instants1); } else { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CommitMetadataSerDe.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CommitMetadataSerDe.java new file mode 100644 index 000000000000..ba2cb57db72a --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CommitMetadataSerDe.java @@ -0,0 +1,35 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.util.Option; + +import java.io.IOException; +import java.io.Serializable; + +/** + * Interface for serializing and deserializing commit metadata. + */ +public interface CommitMetadataSerDe extends Serializable { + + T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException; + + Option serialize(HoodieCommitMetadata commitMetadata) throws IOException; +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CompletionTimeQueryView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CompletionTimeQueryView.java index af4fea7a3bad..8e9f229030eb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CompletionTimeQueryView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/CompletionTimeQueryView.java @@ -18,126 +18,30 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.InstantRange; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.VisibleForTesting; -import org.apache.avro.generic.GenericRecord; - -import java.io.Serializable; -import java.time.Instant; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; import java.util.List; -import java.util.Map; -import java.util.concurrent.ConcurrentHashMap; -import java.util.concurrent.ConcurrentMap; import java.util.function.Function; -import java.util.stream.Collectors; - -import static org.apache.hudi.common.table.read.IncrementalQueryAnalyzer.START_COMMIT_EARLIEST; -import static org.apache.hudi.common.table.timeline.HoodieArchivedTimeline.COMPLETION_TIME_ARCHIVED_META_FIELD; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; - -/** - * Query view for instant completion time. - */ -public class CompletionTimeQueryView implements AutoCloseable, Serializable { - private static final long serialVersionUID = 1L; - - private static final long MILLI_SECONDS_IN_THREE_DAYS = 3 * 24 * 3600 * 1000; - - private static final long MILLI_SECONDS_IN_ONE_DAY = 24 * 3600 * 1000; - - private static final Function GET_INSTANT_ONE_DAY_BEFORE = instant -> - HoodieInstantTimeGenerator.instantTimeMinusMillis(instant, MILLI_SECONDS_IN_ONE_DAY); - private final HoodieTableMetaClient metaClient; - - /** - * Mapping from instant time -> completion time. - * Should be thread-safe data structure. - */ - private final ConcurrentMap instantTimeToCompletionTimeMap; - - /** - * The cursor instant time to eagerly load from, by default load last N days of completed instants. - * It can grow dynamically with lazy loading. e.g. assuming an initial cursor instant as t10, - * a completion query for t5 would trigger lazy loading with this cursor instant updated to t5. - * This sliding window model amortizes redundant loading from different queries. - */ - private volatile String cursorInstant; +public interface CompletionTimeQueryView extends AutoCloseable { - /** - * The first write instant on the active timeline, used for query optimization. - */ - private final String firstNonSavepointCommit; - - /** - * The constructor. - * - * @param metaClient The table meta client. - */ - public CompletionTimeQueryView(HoodieTableMetaClient metaClient) { - this(metaClient, HoodieInstantTimeGenerator.formatDate(new Date(Instant.now().minusMillis(MILLI_SECONDS_IN_THREE_DAYS).toEpochMilli()))); - } - - /** - * The constructor. - * - * @param metaClient The table meta client. - * @param eagerLoadInstant The earliest instant time to eagerly load from, by default load last N days of completed instants. - */ - public CompletionTimeQueryView(HoodieTableMetaClient metaClient, String eagerLoadInstant) { - this.metaClient = metaClient; - this.instantTimeToCompletionTimeMap = new ConcurrentHashMap<>(); - this.cursorInstant = HoodieTimeline.minInstant(eagerLoadInstant, metaClient.getActiveTimeline().firstInstant().map(HoodieInstant::getTimestamp).orElse("")); - // Note: use getWriteTimeline() to keep sync with the fs view visibleCommitsAndCompactionTimeline, see AbstractTableFileSystemView.refreshTimeline. - this.firstNonSavepointCommit = metaClient.getActiveTimeline().getWriteTimeline().getFirstNonSavepointCommit().map(HoodieInstant::getTimestamp).orElse(""); - load(); - } - - /** - * Returns whether the instant is completed. - */ - public boolean isCompleted(String instantTime) { - // archival does not proceed beyond the first savepoint, so any instant before that is completed. - return this.instantTimeToCompletionTimeMap.containsKey(instantTime) || isArchived(instantTime); - } + public boolean isCompleted(String beginInstantTime); /** * Returns whether the instant is archived. */ - public boolean isArchived(String instantTime) { - return HoodieTimeline.compareTimestamps(instantTime, LESSER_THAN, this.firstNonSavepointCommit); - } + public boolean isArchived(String instantTime); /** * Returns whether the give instant time {@code instantTime} completed before the base instant {@code baseInstant}. */ - public boolean isCompletedBefore(String baseInstant, String instantTime) { - Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); - if (completionTimeOpt.isPresent()) { - return HoodieTimeline.compareTimestamps(completionTimeOpt.get(), LESSER_THAN, baseInstant); - } - return false; - } + public boolean isCompletedBefore(String baseInstant, String instantTime); /** * Returns whether the given instant time {@code instantTime} is sliced after or on the base instant {@code baseInstant}. */ - public boolean isSlicedAfterOrOn(String baseInstant, String instantTime) { - Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); - if (completionTimeOpt.isPresent()) { - return HoodieTimeline.compareTimestamps(completionTimeOpt.get(), GREATER_THAN_OR_EQUALS, baseInstant); - } - return true; - } + public boolean isSlicedAfterOrOn(String baseInstant, String instantTime); /** * Get completion time with a base instant time as a reference to fix the compatibility. @@ -147,24 +51,7 @@ public boolean isSlicedAfterOrOn(String baseInstant, String instantTime) { * * @return Probability fixed completion time. */ - public Option getCompletionTime(String baseInstant, String instantTime) { - Option completionTimeOpt = getCompletionTime(instantTime); - if (completionTimeOpt.isPresent()) { - String completionTime = completionTimeOpt.get(); - if (completionTime.length() != baseInstant.length()) { - // ============================================================== - // LEGACY CODE - // ============================================================== - // Fixes the completion time to reflect the completion sequence correctly - // if the file slice base instant time is not in datetime format. - // For example, many test cases just use integer string as the instant time. - // CAUTION: this fix only works for OCC(Optimistic Concurrency Control). - // for NB-CC(Non-blocking Concurrency Control), the file slicing may be incorrect. - return Option.of(instantTime); - } - } - return completionTimeOpt; - } + public Option getCompletionTime(String baseInstant, String instantTime); /** * Queries the completion time with given instant time. @@ -173,18 +60,7 @@ public Option getCompletionTime(String baseInstant, String instantTime) * * @return The completion time if the instant finished or empty if it is still pending. */ - public Option getCompletionTime(String instantTime) { - String completionTime = this.instantTimeToCompletionTimeMap.get(instantTime); - if (completionTime != null) { - return Option.of(completionTime); - } - if (HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, this.cursorInstant)) { - // the instant is still pending - return Option.empty(); - } - loadCompletionTimeIncrementally(instantTime); - return Option.ofNullable(this.instantTimeToCompletionTimeMap.get(instantTime)); - } + public Option getCompletionTime(String beginTime); /** * Queries the instant times with given completion time range. @@ -202,11 +78,7 @@ public List getInstantTimes( HoodieTimeline timeline, Option startCompletionTime, Option endCompletionTime, - InstantRange.RangeType rangeType) { - // assumes any instant/transaction lasts at most 1 day to optimize the query efficiency. - return getInstantTimes( - timeline, startCompletionTime, endCompletionTime, rangeType, GET_INSTANT_ONE_DAY_BEFORE); - } + InstantRange.RangeType rangeType); /** * Queries the instant times with given completion time range. @@ -218,147 +90,20 @@ public List getInstantTimes( * * @return The sorted instant time list. */ - @VisibleForTesting public List getInstantTimes( String startCompletionTime, String endCompletionTime, - Function earliestInstantTimeFunc) { - return getInstantTimes( - metaClient.getCommitsTimeline().filterCompletedInstants(), - Option.ofNullable(startCompletionTime), - Option.ofNullable(endCompletionTime), - InstantRange.RangeType.CLOSED_CLOSED, - earliestInstantTimeFunc); - } + Function earliestInstantTimeFunc); /** - * Queries the instant times with given completion time range. - * - * @param timeline The timeline. - * @param startCompletionTime The start completion time of the query range. - * @param endCompletionTime The end completion time of the query range. - * @param rangeType The range type. - * @param earliestInstantTimeFunc The function to generate the earliest instant time boundary - * with the minimum completion time. - * - * @return The sorted instant time list. + * Get Cursor Instant + * @return */ - public List getInstantTimes( - HoodieTimeline timeline, - Option startCompletionTime, - Option endCompletionTime, - InstantRange.RangeType rangeType, - Function earliestInstantTimeFunc) { - boolean startFromEarliest = START_COMMIT_EARLIEST.equalsIgnoreCase(startCompletionTime.orElse(null)); - String earliestInstantToLoad = null; - if (startCompletionTime.isPresent() && !startFromEarliest) { - earliestInstantToLoad = earliestInstantTimeFunc.apply(startCompletionTime.get()); - } else if (endCompletionTime.isPresent()) { - earliestInstantToLoad = earliestInstantTimeFunc.apply(endCompletionTime.get()); - } - - // ensure the earliest instant boundary be loaded. - if (earliestInstantToLoad != null && HoodieTimeline.compareTimestamps(this.cursorInstant, GREATER_THAN, earliestInstantToLoad)) { - loadCompletionTimeIncrementally(earliestInstantToLoad); - } - - if (startCompletionTime.isEmpty() && endCompletionTime.isPresent()) { - // returns the last instant that finished at or before the given completion time 'endTime'. - String maxInstantTime = timeline.getInstantsAsStream() - .filter(instant -> instant.isCompleted() - && HoodieTimeline.compareTimestamps(instant.getCompletionTime(), LESSER_THAN_OR_EQUALS, endCompletionTime.get())) - .max(Comparator.comparing(HoodieInstant::getCompletionTime)).map(HoodieInstant::getTimestamp).orElse(null); - if (maxInstantTime != null) { - return Collections.singletonList(maxInstantTime); - } - // fallback to archived timeline - return this.instantTimeToCompletionTimeMap.entrySet().stream() - .filter(entry -> HoodieTimeline.compareTimestamps(entry.getValue(), LESSER_THAN_OR_EQUALS, endCompletionTime.get())) - .map(Map.Entry::getKey).collect(Collectors.toList()); - } - - if (startFromEarliest) { - // expedience for snapshot read: ['earliest', _) to avoid loading unnecessary instants. - startCompletionTime = Option.empty(); - } - - if (startCompletionTime.isEmpty() && endCompletionTime.isEmpty()) { - // (_, _): read the latest snapshot. - return timeline.filterCompletedInstants().lastInstant().map(instant -> Collections.singletonList(instant.getTimestamp())).orElse(Collections.emptyList()); - } - - InstantRange instantRange = InstantRange.builder() - .rangeType(rangeType) - .startInstant(startCompletionTime.orElse(null)) - .endInstant(endCompletionTime.orElse(null)) - .nullableBoundary(true) - .build(); - return this.instantTimeToCompletionTimeMap.entrySet().stream() - .filter(entry -> instantRange.isInRange(entry.getValue())) - .map(Map.Entry::getKey).sorted().collect(Collectors.toList()); - } - - // ------------------------------------------------------------------------- - // Utilities - // ------------------------------------------------------------------------- - - private void loadCompletionTimeIncrementally(String startCompletionTime) { - // the 'startCompletionTime' should be out of the eager loading range, switch to a lazy loading. - // This operation is resource costly. - synchronized (this) { - if (HoodieTimeline.compareTimestamps(startCompletionTime, LESSER_THAN, this.cursorInstant)) { - HoodieArchivedTimeline.loadInstants(metaClient, - new HoodieArchivedTimeline.ClosedOpenTimeRangeFilter(startCompletionTime, this.cursorInstant), - HoodieArchivedTimeline.LoadMode.TIME, - r -> true, - this::readCompletionTime); - } - // refresh the start instant - this.cursorInstant = startCompletionTime; - } - } + public String getCursorInstant(); /** - * This is the method to read completion time. - * This would also update 'instantTimeToCompletionTimeMap' map with instant time/completion time pairs. - * Only instants starts from 'startCompletionTime' (inclusive) are considered. + * Return true if the table is empty. + * @return */ - private void load() { - // load active instants first. - this.metaClient.getActiveTimeline() - .filterCompletedInstants().getInstantsAsStream() - .forEach(instant -> setCompletionTime(instant.getTimestamp(), instant.getCompletionTime())); - // then load the archived instants. - HoodieArchivedTimeline.loadInstants(metaClient, - new HoodieArchivedTimeline.StartTsFilter(this.cursorInstant), - HoodieArchivedTimeline.LoadMode.TIME, - r -> true, - this::readCompletionTime); - } - - private void readCompletionTime(String instantTime, GenericRecord record) { - String completionTime = record.get(COMPLETION_TIME_ARCHIVED_META_FIELD).toString(); - setCompletionTime(instantTime, completionTime); - } - - private void setCompletionTime(String instantTime, String completionTime) { - if (completionTime == null) { - // the meta-server instant does not have completion time - completionTime = instantTime; - } - this.instantTimeToCompletionTimeMap.putIfAbsent(instantTime, completionTime); - } - - public String getCursorInstant() { - return cursorInstant; - } - - public boolean isEmptyTable() { - return this.instantTimeToCompletionTimeMap.isEmpty(); - } - - @Override - public void close() { - this.instantTimeToCompletionTimeMap.clear(); - } + public boolean isEmptyTable(); } 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 3eb07d7268d9..37d893517b7f 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 @@ -18,412 +18,148 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.common.model.HoodieCommitMetadata; -import org.apache.hudi.common.model.WriteOperationType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.IOException; -import java.io.InputStream; -import java.io.Serializable; -import java.text.ParseException; -import java.util.Arrays; -import java.util.Collections; -import java.util.Comparator; -import java.util.Date; -import java.util.HashSet; -import java.util.Objects; import java.util.Set; -import java.util.function.Function; -import java.util.stream.Stream; /** - * Represents the Active Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the - * ActiveTimeline and the rest are Archived. ActiveTimeline is a special timeline that allows for creation of instants - * on the timeline. - *

- *

+ * Represents the Active Timeline for the Hoodie table. * The timeline is not automatically reloaded on any mutation operation, clients have to manually call reload() so that * they can chain multiple mutations to the timeline and then call reload() once. *

*

* This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized. */ -public class HoodieActiveTimeline extends HoodieDefaultTimeline { - - public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( - COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, - DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, - SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, - CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, - INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, - REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, - INFLIGHT_LOG_COMPACTION_EXTENSION, REQUESTED_LOG_COMPACTION_EXTENSION, - ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, - REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, - REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION, - REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION, - REQUESTED_CLUSTERING_COMMIT_EXTENSION, INFLIGHT_CLUSTERING_COMMIT_EXTENSION)); - - public static final Set NOT_PARSABLE_TIMESTAMPS = new HashSet(3) {{ - add(HoodieTimeline.INIT_INSTANT_TS); - add(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); - add(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); - }}; - - private static final Logger LOG = LoggerFactory.getLogger(HoodieActiveTimeline.class); - protected HoodieTableMetaClient metaClient; - - /** - * Parse the timestamp of an Instant and return a {@code Date}. - * Throw ParseException if timestamp is not valid format as - * {@link org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. - * - * @param timestamp a timestamp String which follow pattern as - * {@link org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. - * @return Date of instant timestamp - */ - public static Date parseDateFromInstantTime(String timestamp) throws ParseException { - return HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp); - } - - /** - * The same parsing method as above, but this method will mute ParseException. - * If the given timestamp is invalid, returns {@code Option.empty}. - * Or a corresponding Date value if these timestamp strings are provided - * {@link org.apache.hudi.common.table.timeline.HoodieTimeline#INIT_INSTANT_TS}, - * {@link org.apache.hudi.common.table.timeline.HoodieTimeline#METADATA_BOOTSTRAP_INSTANT_TS}, - * {@link org.apache.hudi.common.table.timeline.HoodieTimeline#FULL_BOOTSTRAP_INSTANT_TS}. - * This method is useful when parsing timestamp for metrics - * - * @param timestamp a timestamp String which follow pattern as - * {@link org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. - * @return {@code Option} of instant timestamp, {@code Option.empty} if invalid timestamp - */ - public static Option parseDateFromInstantTimeSafely(String timestamp) { - Option parsedDate; - try { - parsedDate = Option.of(HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp)); - } catch (ParseException e) { - if (NOT_PARSABLE_TIMESTAMPS.contains(timestamp)) { - parsedDate = Option.of(new Date(Integer.parseInt(timestamp))); - } else { - LOG.warn("Failed to parse timestamp " + timestamp + ": " + e.getMessage()); - parsedDate = Option.empty(); - } - } - return parsedDate; - } - - /** - * Format the Date to a String representing the timestamp of a Hoodie Instant. - */ - public static String formatDate(Date timestamp) { - return HoodieInstantTimeGenerator.formatDate(timestamp); - } - - /** - * Returns next instant time in the correct format. - * Ensures each instant time is at least 1 millisecond apart since we create instant times at millisecond granularity. - * - * @param shouldLock whether the lock should be enabled to get the instant time. - * @param timeGenerator TimeGenerator used to generate the instant time. - */ - public static String createNewInstantTime(boolean shouldLock, TimeGenerator timeGenerator) { - return createNewInstantTime(shouldLock, timeGenerator, 0L); - } +public interface HoodieActiveTimeline extends HoodieTimeline { /** - * Returns next instant time in the correct format. - * Ensures each instant time is at least 1 millisecond apart since we create instant times at millisecond granularity. - * - * @param shouldLock whether the lock should be enabled to get the instant time. - * @param timeGenerator TimeGenerator used to generate the instant time. - * @param milliseconds Milliseconds to add to current time while generating the new instant time - */ - public static String createNewInstantTime(boolean shouldLock, TimeGenerator timeGenerator, long milliseconds) { - return HoodieInstantTimeGenerator.createNewInstantTime(shouldLock, timeGenerator, milliseconds); - } - - protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set includedExtensions) { - this(metaClient, includedExtensions, true); - } - - protected HoodieActiveTimeline(HoodieTableMetaClient metaClient, Set includedExtensions, - boolean applyLayoutFilters) { - // Filter all the filter in the metapath and include only the extensions passed and - // convert them into HoodieInstant - try { - this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(includedExtensions, applyLayoutFilters)); - } catch (IOException e) { - throw new HoodieIOException("Failed to scan metadata", e); - } - this.metaClient = metaClient; - // multiple casts will make this lambda serializable - - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; - - LOG.info("Loaded instants upto : " + lastInstant()); - } - - public HoodieActiveTimeline(HoodieTableMetaClient metaClient) { - this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE)); - } - - public HoodieActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { - this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), applyLayoutFilter); - } - - /** - * For serialization and de-serialization only. - * - * @deprecated + * Return Valid extensions expected in active timeline. + * @return */ - @Deprecated - public HoodieActiveTimeline() { - } - - /** - * This method is only used when this object is deserialized in a spark executor. - * - * @deprecated - */ - @Deprecated - private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - } + public Set getValidExtensionsInActiveTimeline(); /** * Create a complete instant and save to storage with a completion time. * @param instant the complete instant. */ - public void createCompleteInstant(HoodieInstant instant) { - LOG.info("Creating a new complete instant " + instant); - createCompleteFileInMetaPath(true, instant, Option.empty()); - } + public void createCompleteInstant(HoodieInstant instant); /** * Create a pending instant and save to storage. * @param instant the pending instant. */ - public void createNewInstant(HoodieInstant instant) { - LOG.info("Creating a new instant " + instant); - ValidationUtils.checkArgument(!instant.isCompleted()); - createFileInMetaPath(instant.getFileName(), Option.empty(), false); - } - - public void createRequestedCommitWithReplaceMetadata(String instantTime, String actionType) { - try { - HoodieInstant instant = new HoodieInstant(State.REQUESTED, actionType, instantTime); - LOG.info("Creating a new instant " + instant); - // Create the request replace file - createFileInMetaPath(instant.getFileName(), - TimelineMetadataUtils.serializeRequestedReplaceMetadata(new HoodieRequestedReplaceMetadata()), false); - } catch (IOException e) { - throw new HoodieIOException("Error create requested replace commit ", e); - } - } - - public void saveAsComplete(HoodieInstant instant, Option data) { - saveAsComplete(true, instant, data); - } - - public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option data) { - LOG.info("Marking instant complete " + instant); - ValidationUtils.checkArgument(instant.isInflight(), - "Could not mark an already completed instant as complete again " + instant); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp()); - transitionStateToComplete(shouldLock, instant, commitInstant, data); - LOG.info("Completed " + instant); - } - - public HoodieInstant revertToInflight(HoodieInstant instant) { - LOG.info("Reverting instant to inflight " + instant); - HoodieInstant inflight = HoodieTimeline.getInflightInstant(instant, metaClient); - revertCompleteToInflight(instant, inflight); - LOG.info("Reverted " + instant + " to inflight " + inflight); - return inflight; - } - - public void deleteInflight(HoodieInstant instant) { - ValidationUtils.checkArgument(instant.isInflight()); - deleteInstantFile(instant); - } - - public void deletePending(HoodieInstant instant) { - ValidationUtils.checkArgument(!instant.isCompleted()); - deleteInstantFile(instant); - } - - public void deleteCompletedRollback(HoodieInstant instant) { - ValidationUtils.checkArgument(instant.isCompleted()); - ValidationUtils.checkArgument(Objects.equals(instant.getAction(), HoodieTimeline.ROLLBACK_ACTION)); - deleteInstantFile(instant); - } - - public static void deleteInstantFile(HoodieStorage storage, StoragePath metaPath, HoodieInstant instant) { - try { - storage.deleteFile(new StoragePath(metaPath, instant.getFileName())); - } catch (IOException e) { - throw new HoodieIOException("Could not delete instant file" + instant.getFileName(), e); - } - } - - public void deleteEmptyInstantIfExists(HoodieInstant instant) { - ValidationUtils.checkArgument(isEmpty(instant)); - deleteInstantFileIfExists(instant); - } - - public void deleteCompactionRequested(HoodieInstant instant) { - ValidationUtils.checkArgument(instant.isRequested()); - ValidationUtils.checkArgument(Objects.equals(instant.getAction(), HoodieTimeline.COMPACTION_ACTION)); - deleteInstantFile(instant); - } + public void createNewInstant(HoodieInstant instant); + + public void createRequestedCommitWithReplaceMetadata(String instantTime, String actionType); /** - * Note: This method should only be used in the case that delete requested/inflight instant or empty clean instant, - * and completed commit instant in an archive operation. + * Save Completed instant in active timeline. + * @param instant Instant to be saved. + * @param data Metadata to be written in the instant file. */ - public void deleteInstantFileIfExists(HoodieInstant instant) { - LOG.info("Deleting instant " + instant); - StoragePath commitFilePath = getInstantFileNamePath(instant.getFileName()); - try { - if (metaClient.getStorage().exists(commitFilePath)) { - boolean result = metaClient.getStorage().deleteFile(commitFilePath); - if (result) { - LOG.info("Removed instant " + instant); - } else { - throw new HoodieIOException("Could not delete instant " + instant + " with path " + commitFilePath); - } - } else { - LOG.warn("The commit " + commitFilePath + " to remove does not exist"); - } - } catch (IOException e) { - throw new HoodieIOException("Could not remove commit " + commitFilePath, e); - } - } - - protected void deleteInstantFile(HoodieInstant instant) { - LOG.info("Deleting instant " + instant); - StoragePath filePath = getInstantFileNamePath(instant.getFileName()); - try { - boolean result = metaClient.getStorage().deleteFile(filePath); - if (result) { - LOG.info("Removed instant " + instant); - } else { - throw new HoodieIOException("Could not delete instant " + instant + " with path " + filePath); - } - } catch (IOException e) { - throw new HoodieIOException("Could not remove inflight commit " + filePath, e); - } - } - - /** - * Many callers might not pass completionTime, here we have to search - * timeline to get completionTime, the impact should be minor since - * 1. It appeals only tests pass instant without completion time - * 2. we already holds all instants in memory, the cost should be minor. - * - *

TODO: [HUDI-6885] Depreciate HoodieActiveTimeline#getInstantFileName and fix related tests. + public void saveAsComplete(HoodieInstant instant, Option data); + + /** + * Save Completed instant in active timeline. + * @param shouldLock Lock before writing to timeline. + * @param instant Instant to be saved. + * @param data Metadata to be written in the instant file. + */ + public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option data); + + /** + * Delete Compaction requested instant file from timeline. + * @param instant Instant to be deleted. */ - protected String getInstantFileName(HoodieInstant instant) { - if (instant.isCompleted() && instant.getCompletionTime() == null) { - return getInstantsAsStream().filter(s -> s.equals(instant)) - .findFirst().orElseThrow(() -> new HoodieIOException("Cannot find the instant" + instant)) - .getFileName(); - } - return instant.getFileName(); - } + public HoodieInstant revertToInflight(HoodieInstant instant); - @Override - public Option getInstantDetails(HoodieInstant instant) { - StoragePath detailPath = getInstantFileNamePath(getInstantFileName(instant)); - return readDataFromPath(detailPath); - } + /** + * Delete inflight instant file from timeline. + * @param instant Instant to be deleted. + */ + public void deleteInflight(HoodieInstant instant); + + /** + * Delete pending instant file from timeline. + * @param instant Instant to be deleted. + */ + public void deletePending(HoodieInstant instant); + + /** + * Delete completed rollback instant file from timeline. + * @param instant Instant to be deleted. + */ + public void deleteCompletedRollback(HoodieInstant instant); + + /** + * Delete empty instant file from timeline. + * @param instant Instant to be deleted. + */ + public void deleteEmptyInstantIfExists(HoodieInstant instant); + + /** + * Delete Compaction requested instant file from timeline. + * @param instant Instant to be deleted. + */ + public void deleteCompactionRequested(HoodieInstant instant); + + /** + * Note: This method should only be used in the case that delete requested/inflight instant or empty clean instant, + * and completed commit instant in an archive operation. + */ + public void deleteInstantFileIfExists(HoodieInstant instant); /** * Returns most recent instant having valid schema in its {@link HoodieCommitMetadata} */ - public Option> getLastCommitMetadataWithValidSchema() { - return Option.fromJavaOptional( - getCommitMetadataStream() - .filter(instantCommitMetadataPair -> - WriteOperationType.canUpdateSchema(instantCommitMetadataPair.getRight().getOperationType()) - && !StringUtils.isNullOrEmpty(instantCommitMetadataPair.getValue().getMetadata(HoodieCommitMetadata.SCHEMA_KEY))) - .findFirst() - ); - } + public Option> getLastCommitMetadataWithValidSchema(); /** * Get the last instant with valid data, and convert this to HoodieCommitMetadata */ - public Option> getLastCommitMetadataWithValidData() { - return Option.fromJavaOptional( - getCommitMetadataStream() - .filter(instantCommitMetadataPair -> - !instantCommitMetadataPair.getValue().getFileIdAndRelativePaths().isEmpty()) - .findFirst() - ); - } - - /** - * Returns stream of {@link HoodieCommitMetadata} in order reverse to chronological (ie most - * recent metadata being the first element) - */ - private Stream> getCommitMetadataStream() { - // NOTE: Streams are lazy - return getCommitsTimeline().filterCompletedInstants() - .getInstantsAsStream() - .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()) - .map(instant -> { - try { - HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(getInstantDetails(instant).get(), HoodieCommitMetadata.class); - return Pair.of(instant, commitMetadata); - } catch (IOException e) { - throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); - } - }); - } - - public Option readCleanerInfoAsBytes(HoodieInstant instant) { - // Cleaner metadata are always stored only in timeline .hoodie - return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); - } - - public Option readRollbackInfoAsBytes(HoodieInstant instant) { - // Rollback metadata are always stored only in timeline .hoodie - return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); - } - - public Option readRestoreInfoAsBytes(HoodieInstant instant) { - // Rollback metadata are always stored only in timeline .hoodie - return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); - } + public Option> getLastCommitMetadataWithValidData(); + + /** + * Read cleaner Info from instant file. + * @param instant Instant to read from. + * @return + */ + public Option readCleanerInfoAsBytes(HoodieInstant instant); + + /** + * Read rollback info from instant file. + * @param instant Instant to read from. + * @return + */ + public Option readRollbackInfoAsBytes(HoodieInstant instant); + + /** + * Read Restore info from instant file. + * @param instant Instant to read from. + * @return + */ + public Option readRestoreInfoAsBytes(HoodieInstant instant); //----------------------------------------------------------------- // BEGIN - COMPACTION RELATED META-DATA MANAGEMENT. //----------------------------------------------------------------- - public Option readCompactionPlanAsBytes(HoodieInstant instant) { - return readDataFromPath(new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant))); - } + /** + * Read compaction Plan from instant file. + * @param instant Instant to read from. + * @return + */ + public Option readCompactionPlanAsBytes(HoodieInstant instant); - public Option readIndexPlanAsBytes(HoodieInstant instant) { - return readDataFromPath(new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant))); - } + /** + * Read Index Plan from instant file. + * @param instant Instant to read from. + * @return + */ + public Option readIndexPlanAsBytes(HoodieInstant instant); /** * Revert instant state from inflight to requested. @@ -431,18 +167,7 @@ public Option readIndexPlanAsBytes(HoodieInstant instant) { * @param inflightInstant Inflight Instant * @return requested instant */ - public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflightInstant) { - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant requestedInstant = - new HoodieInstant(State.REQUESTED, inflightInstant.getAction(), inflightInstant.getTimestamp()); - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - // Pass empty data since it is read from the corresponding .aux/.compaction instant file - transitionPendingState(inflightInstant, requestedInstant, Option.empty()); - } else { - deleteInflight(inflightInstant); - } - return requestedInstant; - } + public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflightInstant); /** * TODO: This method is not needed, since log compaction plan is not a immutable plan. @@ -451,19 +176,7 @@ public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflight * @param inflightInstant Inflight Instant * @return requested instant */ - public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant requestedInstant = - new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, inflightInstant.getTimestamp()); - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - // Pass empty data since it is read from the corresponding .aux/.compaction instant file - transitionPendingState(inflightInstant, requestedInstant, Option.empty()); - } else { - deleteInflight(inflightInstant); - } - return requestedInstant; - } + public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant); /** * Transition Compaction State from requested to inflight. @@ -471,14 +184,7 @@ public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflig * @param requestedInstant Requested instant * @return inflight instant */ - public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflightInstant = - new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflightInstant, Option.empty()); - return inflightInstant; - } + public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant); /** * Transition LogCompaction State from requested to inflight. @@ -486,14 +192,7 @@ public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant reque * @param requestedInstant Requested instant * @return inflight instant */ - public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflightInstant = - new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflightInstant, Option.empty()); - return inflightInstant; - } + public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant); /** * Transition Compaction State from inflight to Committed. @@ -504,13 +203,7 @@ public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant re * @return commit instant */ public HoodieInstant transitionCompactionInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, - Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, COMMIT_ACTION, inflightInstant.getTimestamp()); - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + Option data); /** * Transition Log Compaction State from inflight to Committed. @@ -521,13 +214,7 @@ public HoodieInstant transitionCompactionInflightToComplete(boolean shouldLock, * @return commit instant */ public HoodieInstant transitionLogCompactionInflightToComplete(boolean shouldLock, - HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, DELTA_COMMIT_ACTION, inflightInstant.getTimestamp()); - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + HoodieInstant inflightInstant, Option data); //----------------------------------------------------------------- // END - COMPACTION RELATED META-DATA MANAGEMENT @@ -542,14 +229,7 @@ public HoodieInstant transitionLogCompactionInflightToComplete(boolean shouldLoc * @return commit instant */ public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, - Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, CLEAN_ACTION, inflightInstant.getTimestamp()); - // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + Option data); /** * Transition Clean State from requested to inflight. @@ -558,13 +238,7 @@ public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, Hoodi * @param data Optional data to be stored * @return commit instant */ - public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option data) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflight, data); - return inflight; - } + public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option data); /** * Transition Rollback State from inflight to Committed. @@ -575,14 +249,7 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI * @return commit instant */ public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, - HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, inflightInstant.getTimestamp()); - // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + HoodieInstant inflightInstant, Option data); /** * Transition Rollback State from requested to inflight. @@ -590,13 +257,7 @@ public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, * @param requestedInstant requested instant * @return commit instant */ - public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, ROLLBACK_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflight, Option.empty()); - return inflight; - } + public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant); /** * Transition Restore State from requested to inflight. @@ -604,14 +265,7 @@ public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant request * @param requestedInstant requested instant * @return commit instant */ - public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requestedInstant) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.RESTORE_ACTION), "Transition to inflight requested for a restore instant with diff action " - + requestedInstant); - ValidationUtils.checkArgument(requestedInstant.isRequested(), "Transition to inflight requested for an instant not in requested state " + requestedInstant.toString()); - HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, RESTORE_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflight, Option.empty()); - return inflight; - } + public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requestedInstant); /** * Transition replace requested file to replace inflight. @@ -620,14 +274,7 @@ public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requeste * @param data Extra Metadata * @return inflight instant */ - public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.getTimestamp()); - // Then write to timeline - transitionPendingState(requestedInstant, inflightInstant, data); - return inflightInstant; - } + public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data); /** * Transition cluster requested file to cluster inflight. @@ -636,14 +283,7 @@ public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requeste * @param data Extra Metadata * @return inflight instant */ - public HoodieInstant transitionClusterRequestedToInflight(HoodieInstant requestedInstant, Option data) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested()); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, CLUSTERING_ACTION, requestedInstant.getTimestamp()); - // Then write to timeline - transitionPendingState(requestedInstant, inflightInstant, data); - return inflightInstant; - } + public HoodieInstant transitionClusterRequestedToInflight(HoodieInstant requestedInstant, Option data); /** * Transition replace inflight to Committed. @@ -654,14 +294,7 @@ public HoodieInstant transitionClusterRequestedToInflight(HoodieInstant requeste * @return commit instant */ public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, - HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp()); - // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + HoodieInstant inflightInstant, Option data); /** * Transition cluster inflight to replace committed. @@ -672,193 +305,94 @@ public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, * @return commit instant */ public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, - HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight()); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp()); - // Then write to timeline - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } - - private void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { - transitionPendingState(fromInstant, toInstant, data, false); - } - - protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, - HoodieInstant toInstant, Option data) { - ValidationUtils.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); - String fromInstantFileName = fromInstant.getFileName(); - // Ensures old state exists in timeline - LOG.info("Checking for file exists ?" + getInstantFileNamePath(fromInstantFileName)); - try { - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - // Re-create the .inflight file by opening a new file and write the commit metadata in - createFileInMetaPath(fromInstantFileName, data, false); - StoragePath fromInstantPath = getInstantFileNamePath(fromInstantFileName); - HoodieInstant instantWithCompletionTime = - new HoodieInstant(toInstant.getState(), toInstant.getAction(), - toInstant.getTimestamp(), metaClient.createNewInstantTime(false)); - StoragePath toInstantPath = - getInstantFileNamePath(instantWithCompletionTime.getFileName()); - boolean success = metaClient.getStorage().rename(fromInstantPath, toInstantPath); - if (!success) { - throw new HoodieIOException( - "Could not rename " + fromInstantPath + " to " + toInstantPath); - } - } else { - ValidationUtils.checkArgument( - metaClient.getStorage().exists(getInstantFileNamePath(fromInstantFileName))); - createCompleteFileInMetaPath(shouldLock, toInstant, data); - } - } catch (IOException e) { - throw new HoodieIOException("Could not complete " + fromInstant, e); - } - } - - protected void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, - boolean allowRedundantTransitions) { - ValidationUtils.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); - String fromInstantFileName = fromInstant.getFileName(); - String toInstantFileName = toInstant.getFileName(); - try { - HoodieStorage storage = metaClient.getStorage(); - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - // Re-create the .inflight file by opening a new file and write the commit metadata in - createFileInMetaPath(fromInstantFileName, data, allowRedundantTransitions); - StoragePath fromInstantPath = getInstantFileNamePath(fromInstantFileName); - StoragePath toInstantPath = getInstantFileNamePath(toInstantFileName); - boolean success = storage.rename(fromInstantPath, toInstantPath); - if (!success) { - throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath); - } - } else { - // Ensures old state exists in timeline - ValidationUtils.checkArgument(storage.exists(getInstantFileNamePath(fromInstantFileName)), - "File " + getInstantFileNamePath(fromInstantFileName) + " does not exist!"); - // Use Write Once to create Target File - if (allowRedundantTransitions) { - FileIOUtils.createFileInPath(storage, getInstantFileNamePath(toInstantFileName), data); - } else { - storage.createImmutableFileInPath(getInstantFileNamePath(toInstantFileName), data); - } - LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(toInstantFileName)); - } - } catch (IOException e) { - throw new HoodieIOException("Could not complete " + fromInstant, e); - } - } - - protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { - ValidationUtils.checkArgument(completed.isCompleted()); - ValidationUtils.checkArgument(inflight.isInflight()); - ValidationUtils.checkArgument(completed.getTimestamp().equals(inflight.getTimestamp())); - StoragePath inflightFilePath = getInstantFileNamePath(inflight.getFileName()); - StoragePath completedFilePath = getInstantFileNamePath(getInstantFileName(completed)); - try { - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - if (!metaClient.getStorage().exists(inflightFilePath)) { - boolean success = metaClient.getStorage().rename(completedFilePath, inflightFilePath); - if (!success) { - throw new HoodieIOException( - "Could not rename " + completedFilePath + " to " + inflightFilePath); - } - } - } else { - StoragePath requestedInstantFilePath = getInstantFileNamePath(new HoodieInstant(State.REQUESTED, inflight.getAction(), - inflight.getTimestamp()).getFileName()); - - // If inflight and requested files do not exist, create one - if (!metaClient.getStorage().exists(requestedInstantFilePath)) { - metaClient.getStorage().create(requestedInstantFilePath, false).close(); - } - - if (!metaClient.getStorage().exists(inflightFilePath)) { - metaClient.getStorage().create(inflightFilePath, false).close(); - } - - boolean success = metaClient.getStorage().deleteFile(completedFilePath); - ValidationUtils.checkArgument(success, "State Reverting failed"); - } - } catch (IOException e) { - throw new HoodieIOException("Could not complete revert " + completed, e); - } - } - - private StoragePath getInstantFileNamePath(String fileName) { - return new StoragePath(fileName.contains(SCHEMA_COMMIT_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath().toString(), fileName); - } - - public void transitionRequestedToInflight(String commitType, String inFlightInstant) { - HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant); - transitionRequestedToInflight(requested, Option.empty(), false); - } - - public void transitionRequestedToInflight(HoodieInstant requested, Option content) { - transitionRequestedToInflight(requested, content, false); - } + HoodieInstant inflightInstant, Option data); + /** + * Save Restore requested instant with metadata. + * @param commitType Instant type. + * @param inFlightInstant Instant timestamp. + */ + public void transitionRequestedToInflight(String commitType, String inFlightInstant); + + /** + * Save Restore requested instant with metadata. + * @param requested Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void transitionRequestedToInflight(HoodieInstant requested, Option content); + + /** + * Save Restore requested instant with metadata. + * @param requested Instant to save. + * @param content Metadata to be stored in instant file. + */ public void transitionRequestedToInflight(HoodieInstant requested, Option content, - boolean allowRedundantTransitions) { - HoodieInstant inflight = new HoodieInstant(State.INFLIGHT, requested.getAction(), requested.getTimestamp()); - ValidationUtils.checkArgument(requested.isRequested(), "Instant " + requested + " in wrong state"); - transitionPendingState(requested, inflight, content, allowRedundantTransitions); - } + boolean allowRedundantTransitions); - public void saveToCompactionRequested(HoodieInstant instant, Option content) { - saveToCompactionRequested(instant, content, false); - } + /** + * Save Compaction requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void saveToCompactionRequested(HoodieInstant instant, Option content); - public void saveToCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); - createFileInMetaPath(instant.getFileName(), content, overwrite); - } + /** + * Save Compaction requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + * @param overwrite Overwrite existing instant file. + */ + public void saveToCompactionRequested(HoodieInstant instant, Option content, boolean overwrite); - public void saveToLogCompactionRequested(HoodieInstant instant, Option content) { - saveToLogCompactionRequested(instant, content, false); - } + /** + * Save Log Compaction requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void saveToLogCompactionRequested(HoodieInstant instant, Option content); - public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); - createFileInMetaPath(instant.getFileName(), content, overwrite); - } + /** + * Save Log Compaction requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + * @param overwrite Overwrite existing instant file. + */ + public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite); /** - * Saves content for requested REPLACE instant. + * Save pending replace instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. */ - public void saveToPendingReplaceCommit(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); - createFileInMetaPath(instant.getFileName(), content, false); - } + public void saveToPendingReplaceCommit(HoodieInstant instant, Option content); /** - * Saves content for requested CLUSTER instant. + * Save pending cluster instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. */ - public void saveToPendingClusterCommit(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); - createFileInMetaPath(instant.getFileName(), content, false); - } + public void saveToPendingClusterCommit(HoodieInstant instant, Option content); - public void saveToCleanRequested(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); - ValidationUtils.checkArgument(instant.getState().equals(State.REQUESTED)); - // Plan is stored in meta path - createFileInMetaPath(instant.getFileName(), content, false); - } + /** + * Save clean requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void saveToCleanRequested(HoodieInstant instant, Option content); - public void saveToRollbackRequested(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); - ValidationUtils.checkArgument(instant.getState().equals(State.REQUESTED)); - // Plan is stored in meta path - createFileInMetaPath(instant.getFileName(), content, false); - } + /** + * Save rollback requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void saveToRollbackRequested(HoodieInstant instant, Option content); - public void saveToRestoreRequested(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)); - ValidationUtils.checkArgument(instant.getState().equals(State.REQUESTED)); - // Plan is stored in meta path - createFileInMetaPath(instant.getFileName(), content, false); - } + /** + * Save Restore requested instant with metadata. + * @param instant Instant to save. + * @param content Metadata to be stored in instant file. + */ + public void saveToRestoreRequested(HoodieInstant instant, Option content); /** * Transition index instant state from requested to inflight. @@ -866,15 +400,7 @@ public void saveToRestoreRequested(HoodieInstant instant, Option content * @param requestedInstant Inflight Instant * @return inflight instant */ - public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data) { - ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), - String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEXING_ACTION)); - ValidationUtils.checkArgument(requestedInstant.isRequested(), - String.format("Instant %s not in requested state", requestedInstant.getTimestamp())); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, requestedInstant.getTimestamp()); - transitionPendingState(requestedInstant, inflightInstant, data); - return inflightInstant; - } + public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data); /** * Transition index instant state from inflight to completed. @@ -884,90 +410,36 @@ public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedI * @return completed instant */ public HoodieInstant transitionIndexInflightToComplete(boolean shouldLock, - HoodieInstant inflightInstant, Option data) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), - String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight(), - String.format("Instant %s not inflight", inflightInstant.getTimestamp())); - HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, INDEXING_ACTION, inflightInstant.getTimestamp()); - transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); - return commitInstant; - } + HoodieInstant inflightInstant, Option data); /** * Revert index instant state from inflight to requested. * @param inflightInstant Inflight Instant * @return requested instant */ - public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) { - ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), - String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); - ValidationUtils.checkArgument(inflightInstant.isInflight(), - String.format("Instant %s not inflight", inflightInstant.getTimestamp())); - HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, INDEXING_ACTION, inflightInstant.getTimestamp()); - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - transitionPendingState(inflightInstant, requestedInstant, Option.empty()); - } else { - deleteInflight(inflightInstant); - } - return requestedInstant; - } + public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant); /** * Save content for inflight/requested index instant. */ - public void saveToPendingIndexAction(HoodieInstant instant, Option content) { - ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEXING_ACTION), - String.format("%s is not equal to %s action", instant.getAction(), INDEXING_ACTION)); - createFileInMetaPath(instant.getFileName(), content, false); - } - - protected void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { - StoragePath fullPath = getInstantFileNamePath(filename); - if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { - FileIOUtils.createFileInPath(metaClient.getStorage(), fullPath, content); - } else { - metaClient.getStorage().createImmutableFileInPath(fullPath, content); - } - } - - protected void createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option content) { - TimeGenerator timeGenerator = TimeGenerators - .getTimeGenerator(metaClient.getTimeGeneratorConfig(), metaClient.getStorageConf()); - timeGenerator.consumeTimestamp(!shouldLock, currentTimeMillis -> { - String completionTime = HoodieInstantTimeGenerator.formatDate(new Date(currentTimeMillis)); - String fileName = instant.getFileName(completionTime); - StoragePath fullPath = getInstantFileNamePath(fileName); - if (metaClient.getTimelineLayoutVersion().isNullVersion()) { - FileIOUtils.createFileInPath(metaClient.getStorage(), fullPath, content); - } else { - metaClient.getStorage().createImmutableFileInPath(fullPath, content); - } - LOG.info("Created new file for toInstant ?" + fullPath); - }); - } - - protected Option readDataFromPath(StoragePath detailPath) { - try (InputStream is = metaClient.getStorage().open(detailPath)) { - return Option.of(FileIOUtils.readAsByteArray(is)); - } catch (IOException e) { - throw new HoodieIOException("Could not read commit details from " + detailPath, e); - } - } - - public HoodieActiveTimeline reload() { - return new HoodieActiveTimeline(metaClient); - } - - public void copyInstant(HoodieInstant instant, StoragePath dstDir) { - StoragePath srcPath = new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant)); - StoragePath dstPath = new StoragePath(dstDir, getInstantFileName(instant)); - try { - HoodieStorage storage = metaClient.getStorage(); - storage.createDirectory(dstDir); - FileIOUtils.copy(storage, srcPath, storage, dstPath, false, true); - } catch (IOException e) { - throw new HoodieIOException("Could not copy instant from " + srcPath + " to " + dstPath, e); - } - } + public void saveToPendingIndexAction(HoodieInstant instant, Option content); + + /** + * Reloads timeline from storage + * @return + */ + public HoodieActiveTimeline reload(); + + /** + * Copies instant file from active timeline to destination directory. + * @param instant Instant to copy. + * @param dstDir Destination location. + */ + public void copyInstant(HoodieInstant instant, StoragePath dstDir); + + /** + * Valid Extensions in active timeline. + * @return + */ + Set getValidExtensions(); } 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 17a308c9f129..88a9d98c45d1 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 @@ -18,290 +18,28 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; -import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; -import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.collection.ClosableIterator; -import org.apache.hudi.exception.HoodieIOException; -import org.apache.hudi.io.storage.HoodieAvroFileReader; -import org.apache.hudi.io.storage.HoodieIOFactory; -import org.apache.hudi.storage.StoragePath; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; +public interface HoodieArchivedTimeline extends HoodieTimeline { -import javax.annotation.Nullable; - -import java.io.IOException; -import java.io.Serializable; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.List; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.ConcurrentHashMap; -import java.util.function.BiConsumer; -import java.util.function.Function; - -import static org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER; - -/** - * Represents the Archived Timeline for the Hoodie table. - * - *

Timeline Refresh

- *

Instants are read from the archive file during initialization and never refreshed. To refresh, clients need to call - * #reload(). - * - *

Serialization/De-serialization

- *

This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized. - */ -public class HoodieArchivedTimeline extends HoodieDefaultTimeline { - public static final String INSTANT_TIME_ARCHIVED_META_FIELD = "instantTime"; public static final String COMPLETION_TIME_ARCHIVED_META_FIELD = "completionTime"; - private static final String ACTION_ARCHIVED_META_FIELD = "action"; - private static final String METADATA_ARCHIVED_META_FIELD = "metadata"; - private static final String PLAN_ARCHIVED_META_FIELD = "plan"; - private HoodieTableMetaClient metaClient; - private final Map readCommits = new ConcurrentHashMap<>(); - - private static final Logger LOG = LoggerFactory.getLogger(HoodieArchivedTimeline.class); - - /** - * Used for loading the archived timeline incrementally, the earliest loaded instant time get memorized - * each time the timeline is loaded. The instant time is then used as the end boundary - * of the next loading. - */ - private String cursorInstant; - - /** - * Loads all the archived instants. - * Note that there is no lazy loading, so this may not work if the archived timeline range is really long. - * TBD: Should we enforce maximum time range? - */ - public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { - this.metaClient = metaClient; - setInstants(this.loadInstants()); - this.cursorInstant = firstInstant().map(HoodieInstant::getTimestamp).orElse(null); - // multiple casts will make this lambda serializable - - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; - } - - /** - * Loads completed instants from startTs(inclusive). - * Note that there is no lazy loading, so this may not work if really early startTs is specified. - */ - public HoodieArchivedTimeline(HoodieTableMetaClient metaClient, String startTs) { - this.metaClient = metaClient; - setInstants(loadInstants(new StartTsFilter(startTs), LoadMode.METADATA)); - this.cursorInstant = startTs; - // multiple casts will make this lambda serializable - - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 - this.details = (Function> & Serializable) this::getInstantDetails; - } - - /** - * For serialization and de-serialization only. - * - * @deprecated - */ - public HoodieArchivedTimeline() { - } - - /** - * This method is only used when this object is deserialized in a spark executor. - * - * @deprecated - */ - private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { - in.defaultReadObject(); - } - - public void loadInstantDetailsInMemory(String startTs, String endTs) { - loadInstants(startTs, endTs); - } - - public void loadCompletedInstantDetailsInMemory() { - loadInstants(null, LoadMode.METADATA); - } - - public void loadCompactionDetailsInMemory(String compactionInstantTime) { - loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime); - } - public void loadCompactionDetailsInMemory(String startTs, String endTs) { - // load compactionPlan - loadInstants(new TimeRangeFilter(startTs, endTs), LoadMode.PLAN, - record -> record.get(ACTION_ARCHIVED_META_FIELD).toString().equals(HoodieTimeline.COMMIT_ACTION) - && record.get(PLAN_ARCHIVED_META_FIELD) != null - ); - } - - public void clearInstantDetailsFromMemory(String instantTime) { - this.readCommits.remove(instantTime); - } - - public void clearInstantDetailsFromMemory(String startTs, String endTs) { - this.findInstantsInRange(startTs, endTs).getInstants().forEach(instant -> - this.readCommits.remove(instant.getTimestamp())); - } - - @Override - public Option getInstantDetails(HoodieInstant instant) { - return Option.ofNullable(readCommits.get(instant.getTimestamp())); - } - - public HoodieArchivedTimeline reload() { - return new HoodieArchivedTimeline(metaClient); - } - - /** - * Reloads the archived timeline incrementally with given beginning timestamp {@code startTs}. - * This method is not thread safe. - * - *

IMPORTANT: this is for multiple loading of one static snapshot of the timeline, if there is new instants got archived, - * use {@link #reload()} instead. - */ - public HoodieArchivedTimeline reload(String startTs) { - if (this.cursorInstant != null) { - if (HoodieTimeline.compareTimestamps(startTs, LESSER_THAN, this.cursorInstant)) { - appendInstants(loadInstants(new ClosedOpenTimeRangeFilter(startTs, this.cursorInstant), LoadMode.METADATA)); - this.cursorInstant = startTs; - } - return this; - } else { - // a null cursor instant indicates an empty timeline - return new HoodieArchivedTimeline(metaClient, startTs); - } - } - - private HoodieInstant readCommit(String instantTime, GenericRecord record, Option> instantDetailsConsumer) { - final String action = record.get(ACTION_ARCHIVED_META_FIELD).toString(); - final String completionTime = record.get(COMPLETION_TIME_ARCHIVED_META_FIELD).toString(); - instantDetailsConsumer.ifPresent(consumer -> consumer.accept(instantTime, record)); - return new HoodieInstant(HoodieInstant.State.COMPLETED, action, instantTime, completionTime); - } + public void loadInstantDetailsInMemory(String startTs, String endTs); - @Nullable - private BiConsumer getInstantDetailsFunc(LoadMode loadMode) { - switch (loadMode) { - case METADATA: - return (instant, record) -> { - ByteBuffer commitMeta = (ByteBuffer) record.get(METADATA_ARCHIVED_META_FIELD); - if (commitMeta != null) { - // in case the entry comes from an empty completed meta file - this.readCommits.put(instant, commitMeta.array()); - } - }; - case PLAN: - return (instant, record) -> { - ByteBuffer plan = (ByteBuffer) record.get(PLAN_ARCHIVED_META_FIELD); - if (plan != null) { - // in case the entry comes from an empty completed meta file - this.readCommits.put(instant, plan.array()); - } - }; - default: - return null; - } - } - - private List loadInstants() { - return loadInstants(null, LoadMode.ACTION); - } + public void loadCompletedInstantDetailsInMemory(); - private List loadInstants(String startTs, String endTs) { - return loadInstants(new TimeRangeFilter(startTs, endTs), LoadMode.METADATA); - } - - private List loadInstants(TimeRangeFilter filter, LoadMode loadMode) { - return loadInstants(filter, loadMode, r -> true); - } + public void loadCompactionDetailsInMemory(String compactionInstantTime); - /** - * This is method to read selected instants. Do NOT use this directly use one of the helper methods above - * If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details - * If filter is specified, only the filtered instants are loaded - * If commitsFilter is specified, only the filtered records are loaded. - */ - private List loadInstants( - @Nullable TimeRangeFilter filter, - LoadMode loadMode, - Function commitsFilter) { - Map instantsInRange = new ConcurrentHashMap<>(); - Option> instantDetailsConsumer = Option.ofNullable(getInstantDetailsFunc(loadMode)); - loadInstants(metaClient, filter, loadMode, commitsFilter, (instantTime, avroRecord) -> instantsInRange.putIfAbsent(instantTime, readCommit(instantTime, avroRecord, instantDetailsConsumer))); - ArrayList result = new ArrayList<>(instantsInRange.values()); - Collections.sort(result); - return result; - } + public void loadCompactionDetailsInMemory(String startTs, String endTs); - /** - * Loads the instants from the timeline. - * - * @param metaClient The meta client. - * @param filter The time range filter where the target instant belongs to. - * @param loadMode The load mode. - * @param commitsFilter Filter of the instant type. - * @param recordConsumer Consumer of the instant record payload. - */ - public static void loadInstants( - HoodieTableMetaClient metaClient, - @Nullable TimeRangeFilter filter, - LoadMode loadMode, - Function commitsFilter, - BiConsumer recordConsumer) { - try { - // List all files - List fileNames = LSMTimeline.latestSnapshotManifest(metaClient).getFileNames(); + public void clearInstantDetailsFromMemory(String instantTime); - Schema readSchema = LSMTimeline.getReadSchema(loadMode); - fileNames.stream() - .filter(fileName -> filter == null || LSMTimeline.isFileInRange(filter, fileName)) - .parallel().forEach(fileName -> { - // Read the archived file - try (HoodieAvroFileReader reader = (HoodieAvroFileReader) HoodieIOFactory.getIOFactory(metaClient.getStorage()) - .getReaderFactory(HoodieRecordType.AVRO) - .getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, new StoragePath(metaClient.getArchivePath(), fileName))) { - try (ClosableIterator iterator = reader.getIndexedRecordIterator(HoodieLSMTimelineInstant.getClassSchema(), readSchema)) { - while (iterator.hasNext()) { - GenericRecord record = (GenericRecord) iterator.next(); - String instantTime = record.get(INSTANT_TIME_ARCHIVED_META_FIELD).toString(); - if ((filter == null || filter.isInRange(instantTime)) - && commitsFilter.apply(record)) { - recordConsumer.accept(instantTime, record); - } - } - } - } catch (IOException ioException) { - throw new HoodieIOException("Error open file reader for path: " - + new StoragePath(metaClient.getArchivePath(), fileName)); - } - }); - } catch (IOException e) { - throw new HoodieIOException( - "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); - } - } + public void clearInstantDetailsFromMemory(String startTs, String endTs); - @Override - public HoodieDefaultTimeline getWriteTimeline() { - // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION); - return new HoodieDefaultTimeline(getInstantsAsStream().filter(i -> - readCommits.containsKey(i.getTimestamp())) - .filter(s -> validActions.contains(s.getAction())), details); - } + public HoodieArchivedTimeline reload(); - // ------------------------------------------------------------------------- - // Inner Class - // ------------------------------------------------------------------------- + public HoodieArchivedTimeline reload(String startTs); /** * Different mode for loading the archived instant metadata. @@ -338,7 +76,7 @@ public TimeRangeFilter(String startTs, String endTs) { } public boolean isInRange(String instantTime) { - return HoodieTimeline.isInRange(instantTime, this.startTs, this.endTs); + return InstantComparison.isInRange(instantTime, this.startTs, this.endTs); } } @@ -352,7 +90,7 @@ public ClosedOpenTimeRangeFilter(String startTs, String endTs) { } public boolean isInRange(String instantTime) { - return HoodieTimeline.isInClosedOpenRange(instantTime, this.startTs, this.endTs); + return InstantComparison.isInClosedOpenRange(instantTime, this.startTs, this.endTs); } } @@ -366,7 +104,8 @@ public StartTsFilter(String startTs) { } public boolean isInRange(String instantTime) { - return HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, startTs); + return compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, startTs); } } + } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index dc4abb04a8b6..8d874708cbcb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -19,162 +19,44 @@ package org.apache.hudi.common.table.timeline; import org.apache.hudi.common.util.StringUtils; -import org.apache.hudi.common.util.ValidationUtils; -import org.apache.hudi.storage.StoragePathInfo; import java.io.Serializable; import java.util.Comparator; -import java.util.Date; -import java.util.HashMap; -import java.util.Map; import java.util.Objects; -import java.util.regex.Matcher; -import java.util.regex.Pattern; /** * A Hoodie Instant represents a action done on a hoodie table. All actions start with a inflight instant and then * create a completed instant after done. - * - * @see HoodieTimeline */ public class HoodieInstant implements Serializable, Comparable { - // Instant like 20230104152218702.commit.request, 20230104152218702.inflight and 20230104152218702_20230104152630238.commit - private static final Pattern NAME_FORMAT = - Pattern.compile("^(\\d+(_\\d+)?)(\\.\\w+)(\\.\\D+)?$"); - - private static final String DELIMITER = "."; - + public static final String FILE_NAME_FORMAT_ERROR = "The provided file name %s does not conform to the required format"; public static final String UNDERSCORE = "_"; - - private static final String FILE_NAME_FORMAT_ERROR = - "The provided file name %s does not conform to the required format"; - - /** - * A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants - * for state transitions, this needs to be taken into account. - */ - private static final Map COMPARABLE_ACTIONS = createComparableActionsMap(); - - public static final Comparator ACTION_COMPARATOR = - Comparator.comparing(instant -> getComparableAction(instant.getAction())); - - public static final Comparator INSTANT_TIME_COMPARATOR = Comparator.comparing(HoodieInstant::getTimestamp) - .thenComparing(ACTION_COMPARATOR).thenComparing(HoodieInstant::getState); - - public static final Comparator COMPLETION_TIME_COMPARATOR = - Comparator.comparing(HoodieInstant::getCompletionTime) - .thenComparing(INSTANT_TIME_COMPARATOR); - - private static final String EMPTY_FILE_EXTENSION = ""; - - public static String getComparableAction(String action) { - return COMPARABLE_ACTIONS.getOrDefault(action, action); - } - - public static String extractTimestamp(String fileName) throws IllegalArgumentException { - Matcher matcher = NAME_FORMAT.matcher(fileName); - if (matcher.find()) { - String timestamp = matcher.group(1); - return timestamp.contains(UNDERSCORE) ? timestamp.split(UNDERSCORE)[0] : timestamp; - } - - throw new IllegalArgumentException("Failed to retrieve timestamp from name: " - + String.format(FILE_NAME_FORMAT_ERROR, fileName)); - } - - public static String getTimelineFileExtension(String fileName) { - Objects.requireNonNull(fileName); - - Matcher matcher = NAME_FORMAT.matcher(fileName); - if (matcher.find()) { - return fileName.substring(matcher.group(1).length()); - } - - return EMPTY_FILE_EXTENSION; - } - - /** - * Instant State. - */ - public enum State { - // Requested State (valid state for Compaction) - REQUESTED, - // Inflight instant - INFLIGHT, - // Committed instant - COMPLETED, - // Invalid instant - NIL - } + public static final String EMPTY_FILE_EXTENSION = ""; private final State state; private final String action; - private final String timestamp; + private final String requestedTime; private final String completionTime; // Marker for older formats, we need the state transition time (pre table version 7) private boolean isLegacy = false; + private final Comparator comparator; - /** - * Load the instant from the meta FileStatus. - */ - public HoodieInstant(StoragePathInfo pathInfo) { - // First read the instant timestamp. [==>20170101193025<==].commit - String fileName = pathInfo.getPath().getName(); - Matcher matcher = NAME_FORMAT.matcher(fileName); - if (matcher.find()) { - String[] timestamps = matcher.group(1).split(UNDERSCORE); - timestamp = timestamps[0]; - if (matcher.group(3).equals(HoodieTimeline.INFLIGHT_EXTENSION)) { - // This is to support backwards compatibility on how in-flight commit files were written - // General rule is inflight extension is ..inflight, but for commit it is .inflight - action = HoodieTimeline.COMMIT_ACTION; - state = State.INFLIGHT; - } else { - action = matcher.group(3).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING); - if (matcher.groupCount() == 4 && matcher.group(4) != null) { - state = State.valueOf(matcher.group(4).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING).toUpperCase()); - } else { - // Like 20230104152218702.commit - state = State.COMPLETED; - } - } - if (state == State.COMPLETED) { - if (timestamps.length > 1) { - completionTime = timestamps[1]; - } else { - // for backward compatibility with 0.x release. - completionTime = HoodieInstantTimeGenerator.formatDate(new Date(pathInfo.getModificationTime())); - isLegacy = true; - } - } else { - completionTime = null; - } - } else { - throw new IllegalArgumentException("Failed to construct HoodieInstant: " + String.format(FILE_NAME_FORMAT_ERROR, fileName)); - } - } - - public HoodieInstant(boolean isInflight, String action, String timestamp) { - // TODO: vb - Preserving for avoiding cascading changes. This constructor will be updated in subsequent PR - this.state = isInflight ? State.INFLIGHT : State.COMPLETED; - this.action = action; - this.timestamp = timestamp; - this.completionTime = null; + public HoodieInstant(State state, String action, String requestTime, Comparator comparator) { + this(state, action, requestTime, null, comparator); } - public HoodieInstant(State state, String action, String timestamp) { - this.state = state; - this.action = action; - this.timestamp = timestamp; - this.completionTime = null; + public HoodieInstant(State state, String action, String requestTime, String completionTime, Comparator comparator) { + this(state, action, requestTime, completionTime, false, comparator); } - public HoodieInstant(State state, String action, String timestamp, String completionTime) { + public HoodieInstant(State state, String action, String requestedTime, String completionTime, boolean isLegacy, Comparator comparator) { this.state = state; this.action = action; - this.timestamp = timestamp; + this.requestedTime = requestedTime; this.completionTime = completionTime; + this.isLegacy = isLegacy; + this.comparator = comparator; } public boolean isCompleted() { @@ -193,138 +75,12 @@ public String getAction() { return action; } - public String getTimestamp() { - return timestamp; + public String requestedTime() { + return requestedTime; } - private static Map createComparableActionsMap() { - Map comparableMap = new HashMap<>(); - comparableMap.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION); - comparableMap.put(HoodieTimeline.LOG_COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION); - comparableMap.put(HoodieTimeline.CLUSTERING_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION); - return comparableMap; - } - - private String getPendingFileName() { - if (HoodieTimeline.COMMIT_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightCommitFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedCommitFileName(timestamp); - } - } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightCleanerFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedCleanerFileName(timestamp); - } - } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightRollbackFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedRollbackFileName(timestamp); - } - } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { - return HoodieTimeline.makeInflightSavePointFileName(timestamp); - } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightDeltaFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedDeltaFileName(timestamp); - } - } else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightCompactionFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedCompactionFileName(timestamp); - } - } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightLogCompactionFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedLogCompactionFileName(timestamp); - } - } else if (HoodieTimeline.RESTORE_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightRestoreFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedRestoreFileName(timestamp); - } - } else if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightReplaceFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedReplaceFileName(timestamp); - } - } else if (HoodieTimeline.CLUSTERING_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightClusteringFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedClusteringFileName(timestamp); - } - } else if (HoodieTimeline.INDEXING_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightIndexFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestedIndexFileName(timestamp); - } - } else if (HoodieTimeline.SCHEMA_COMMIT_ACTION.equals(action)) { - if (isInflight()) { - return HoodieTimeline.makeInflightSchemaFileName(timestamp); - } else if (isRequested()) { - return HoodieTimeline.makeRequestSchemaFileName(timestamp); - } - } - throw new IllegalArgumentException("Cannot get file name for unknown action " + action); - } - - private String getCompleteFileName(String completionTime) { - ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(completionTime), "Completion time should not be empty"); - String timestampWithCompletionTime = isLegacy ? timestamp : timestamp + "_" + completionTime; - switch (action) { - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.COMPACTION_ACTION: - return HoodieTimeline.makeCommitFileName(timestampWithCompletionTime); - case HoodieTimeline.CLEAN_ACTION: - return HoodieTimeline.makeCleanerFileName(timestampWithCompletionTime); - case HoodieTimeline.ROLLBACK_ACTION: - return HoodieTimeline.makeRollbackFileName(timestampWithCompletionTime); - case HoodieTimeline.SAVEPOINT_ACTION: - return HoodieTimeline.makeSavePointFileName(timestampWithCompletionTime); - case HoodieTimeline.DELTA_COMMIT_ACTION: - case HoodieTimeline.LOG_COMPACTION_ACTION: - return HoodieTimeline.makeDeltaFileName(timestampWithCompletionTime); - case HoodieTimeline.RESTORE_ACTION: - return HoodieTimeline.makeRestoreFileName(timestampWithCompletionTime); - case HoodieTimeline.REPLACE_COMMIT_ACTION: - return HoodieTimeline.makeReplaceFileName(timestampWithCompletionTime); - case HoodieTimeline.INDEXING_ACTION: - return HoodieTimeline.makeIndexCommitFileName(timestampWithCompletionTime); - case HoodieTimeline.SCHEMA_COMMIT_ACTION: - return HoodieTimeline.makeSchemaFileName(timestampWithCompletionTime); - default: - throw new IllegalArgumentException("Cannot get complete instant's file name for unknown action " - + action); - } - } - - /** - * Get the filename for this instant. - */ - public String getFileName() { - if (isCompleted()) { - return getCompleteFileName(completionTime); - } - - return getPendingFileName(); - } - - /** - * Get the filename for this instant. - */ - public String getFileName(String completionTime) { - ValidationUtils.checkState(isCompleted()); - return getCompleteFileName(completionTime); + public boolean isLegacy() { + return isLegacy; } @Override @@ -336,7 +92,7 @@ public boolean equals(Object o) { return false; } HoodieInstant that = (HoodieInstant) o; - return state == that.state && Objects.equals(action, that.action) && Objects.equals(timestamp, that.timestamp); + return state == that.state && Objects.equals(action, that.action) && Objects.equals(requestedTime, that.requestedTime); } public State getState() { @@ -349,19 +105,33 @@ public String getCompletionTime() { @Override public int hashCode() { - return Objects.hash(state, action, timestamp); + return Objects.hash(state, action, requestedTime); } @Override public int compareTo(HoodieInstant o) { - return INSTANT_TIME_COMPARATOR.compare(this, o); + return comparator.compare(this, o); } @Override public String toString() { return "[" + ((isInflight() || isRequested()) ? "==>" : "") - + timestamp + + requestedTime + (StringUtils.isNullOrEmpty(completionTime) ? "" : ("__" + completionTime)) + "__" + action + "__" + state + "]"; } + + /** + * Instant State. + */ + public enum State { + // Requested State (valid state for Compaction) + REQUESTED, + // Inflight instant + INFLIGHT, + // Committed instant + COMPLETED, + // Invalid instant + NIL + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java index bf89be949421..73728de44a62 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantTimeGenerator.java @@ -33,6 +33,9 @@ import java.util.Date; import java.util.concurrent.atomic.AtomicReference; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * Utility class to generate and parse timestamps used in Instants. */ @@ -80,7 +83,7 @@ public static String createNewInstantTime(boolean shouldLock, TimeGenerator time } else { newCommitTime = MILLIS_INSTANT_TIME_FORMATTER.format(convertDateToTemporalAccessor(d)); } - } while (HoodieTimeline.compareTimestamps(newCommitTime, HoodieActiveTimeline.LESSER_THAN_OR_EQUALS, oldVal)); + } while (compareTimestamps(newCommitTime, LESSER_THAN_OR_EQUALS, oldVal)); return newCommitTime; }); } 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 3b45510d1314..33ea149e1485 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 @@ -18,16 +18,14 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import java.io.Serializable; import java.util.List; -import java.util.Objects; -import java.util.function.BiPredicate; +import java.util.Set; import java.util.function.Predicate; import java.util.stream.Stream; @@ -39,7 +37,6 @@ * this can be chained. * * @see HoodieTableMetaClient - * @see HoodieDefaultTimeline * @see HoodieInstant * @since 0.3.0 */ @@ -83,8 +80,8 @@ public interface HoodieTimeline extends Serializable { String REQUESTED_ROLLBACK_EXTENSION = "." + ROLLBACK_ACTION + REQUESTED_EXTENSION; String INFLIGHT_SAVEPOINT_EXTENSION = "." + SAVEPOINT_ACTION + INFLIGHT_EXTENSION; String REQUESTED_COMPACTION_SUFFIX = StringUtils.join(COMPACTION_ACTION, REQUESTED_EXTENSION); - String COMPLETED_COMPACTION_SUFFIX = StringUtils.join(COMPACTION_ACTION, COMPLETED_EXTENSION); String REQUESTED_COMPACTION_EXTENSION = StringUtils.join(".", REQUESTED_COMPACTION_SUFFIX); + String COMPLETED_COMPACTION_SUFFIX = StringUtils.join(COMPACTION_ACTION, COMPLETED_EXTENSION); String INFLIGHT_COMPACTION_EXTENSION = StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION); String REQUESTED_RESTORE_EXTENSION = "." + RESTORE_ACTION + REQUESTED_EXTENSION; String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION; @@ -157,12 +154,13 @@ public interface HoodieTimeline extends Serializable { HoodieTimeline filterCompletedInstants(); // TODO: Check if logcompaction also needs to be included in this API. + /** * Filter this timeline to just include the completed + compaction (inflight + requested) instants A RT filesystem * view is constructed with this timeline so that file-slice after pending compaction-requested instant-time is also * considered valid. A RT file-system view for reading must then merge the file-slices before and after pending * compaction instant so that all delta-commits are read. - * + * * @return New instance of HoodieTimeline with just completed instants */ HoodieTimeline filterCompletedAndCompactionInstants(); @@ -178,7 +176,7 @@ public interface HoodieTimeline extends Serializable { /** * Timeline to just include commits (commit/deltacommit), compaction and replace actions. - * + * * @return */ HoodieTimeline getWriteTimeline(); @@ -201,7 +199,7 @@ public interface HoodieTimeline extends Serializable { /** * Filter this timeline to just include requested and inflight compaction instants. - * + * * @return */ HoodieTimeline filterPendingCompactionTimeline(); @@ -270,7 +268,8 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline findInstantsInClosedRange(String startTs, String endTs); - /**` + /** + * ` * Create a new Timeline with instants after startTs and before or on endTs * by state transition timestamp of actions. */ @@ -305,7 +304,7 @@ public interface HoodieTimeline extends Serializable { * Create new timeline with all instants before or equals specified time. */ HoodieTimeline findInstantsBeforeOrEquals(String instantTime); - + /** * Custom Filter of Instants. */ @@ -338,13 +337,6 @@ public interface HoodieTimeline extends Serializable { */ Option firstInstant(); - /** - * @param action Instant action String. - * @param state Instant State. - * @return first instant of a specific action and state if available - */ - Option firstInstant(String action, State state); - /** * @return nth completed instant from the first completed instant */ @@ -355,10 +347,9 @@ public interface HoodieTimeline extends Serializable { */ Option lastInstant(); - /** * Get hash of timeline. - * + * * @return */ String getTimelineHash(); @@ -380,7 +371,7 @@ public interface HoodieTimeline extends Serializable { /** * @return true if the passed instant is present as a completed instant on the timeline or if the instant is before - * the first completed instant in the timeline + * the first completed instant in the timeline */ boolean containsOrBeforeTimelineStarts(String ts); @@ -394,9 +385,17 @@ public interface HoodieTimeline extends Serializable { */ List getInstants(); + /** + * First instant that matches the action and state + * @param action + * @param state + * @return + */ + Option firstInstant(String action, HoodieInstant.State state); + /** * @return Get the stream of completed instants in reverse order TODO Change code references to getInstants() that - * reverse the instants later on to use this method instead. + * reverse the instants later on to use this method instead. */ Stream getReverseOrderedInstants(); @@ -432,23 +431,22 @@ public interface HoodieTimeline extends Serializable { /** * get the most recent cluster commit if present */ - public Option getLastClusteringInstant(); + Option getLastClusteringInstant(); /** * get the most recent pending cluster commit if present - * */ - public Option getLastPendingClusterInstant(); - + Option getLastPendingClusterInstant(); + /** * get the least recent pending cluster commit if present */ - public Option getFirstPendingClusterInstant(); + Option getFirstPendingClusterInstant(); /** * return true if instant is a pending clustering commit, otherwise false */ - public boolean isPendingClusteringInstant(String instantTime); + boolean isPendingClusteringInstant(String instantTime); /** * Read the completed instant details. @@ -458,305 +456,81 @@ public interface HoodieTimeline extends Serializable { boolean isEmpty(HoodieInstant instant); /** - * Helper methods to compare instants. - **/ - BiPredicate EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) == 0; - BiPredicate GREATER_THAN_OR_EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) >= 0; - BiPredicate GREATER_THAN = (commit1, commit2) -> commit1.compareTo(commit2) > 0; - BiPredicate LESSER_THAN_OR_EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) <= 0; - BiPredicate LESSER_THAN = (commit1, commit2) -> commit1.compareTo(commit2) < 0; - - static boolean compareTimestamps(String commit1, BiPredicate predicateToApply, String commit2) { - return predicateToApply.test(commit1, commit2); - } + * Get all instants (commits, delta commits) that produce new data, in the active timeline. + */ + HoodieTimeline getCommitsTimeline(); /** - * Returns smaller of the two given timestamps. Returns the non null argument if one of the argument is null. + * Get all instants (commits, delta commits, replace, compaction) that produce new data or merge file, in the active timeline. */ - static String minTimestamp(String commit1, String commit2) { - if (StringUtils.isNullOrEmpty(commit1)) { - return commit2; - } else if (StringUtils.isNullOrEmpty(commit2)) { - return commit1; - } - return minInstant(commit1, commit2); - } + HoodieTimeline getCommitsAndCompactionTimeline(); /** - * Returns smaller of the two given instants compared by their respective timestamps. - * Returns the non null argument if one of the argument is null. + * Get all instants (commits, delta commits, compaction, clean, savepoint, rollback, replace commits, index) that result in actions, + * in the active timeline. */ - static HoodieInstant minTimestampInstant(HoodieInstant instant1, HoodieInstant instant2) { - String commit1 = instant1 != null ? instant1.getTimestamp() : null; - String commit2 = instant2 != null ? instant2.getTimestamp() : null; - String minTimestamp = minTimestamp(commit1, commit2); - return Objects.equals(minTimestamp, commit1) ? instant1 : instant2; - } + HoodieTimeline getAllCommitsTimeline(); /** - * Returns the smaller of the given two instants. + * Get only pure commit and replace commits (inflight and completed) in the active timeline. */ - static String minInstant(String instant1, String instant2) { - return compareTimestamps(instant1, LESSER_THAN, instant2) ? instant1 : instant2; - } + HoodieTimeline getCommitAndReplaceTimeline(); /** - * Returns the greater of the given two instants. + * Get only pure commits (inflight and completed) in the active timeline. */ - static String maxInstant(String instant1, String instant2) { - return compareTimestamps(instant1, GREATER_THAN, instant2) ? instant1 : instant2; - } + HoodieTimeline getCommitTimeline(); /** - * Return true if specified timestamp is in range (startTs, endTs]. + * Get only the delta commits (inflight and completed) in the active timeline. */ - static boolean isInRange(String timestamp, String startTs, String endTs) { - return HoodieTimeline.compareTimestamps(timestamp, GREATER_THAN, startTs) - && HoodieTimeline.compareTimestamps(timestamp, LESSER_THAN_OR_EQUALS, endTs); - } + HoodieTimeline getDeltaCommitTimeline(); /** - * Return true if specified timestamp is in range [startTs, endTs). + * Get a timeline of a specific set of actions. useful to create a merged timeline of multiple actions. + * + * @param actions actions allowed in the timeline */ - static boolean isInClosedOpenRange(String timestamp, String startTs, String endTs) { - return HoodieTimeline.compareTimestamps(timestamp, GREATER_THAN_OR_EQUALS, startTs) - && HoodieTimeline.compareTimestamps(timestamp, LESSER_THAN, endTs); - } + HoodieTimeline getTimelineOfActions(Set actions); /** - * Return true if specified timestamp is in range [startTs, endTs]. + * Get only the cleaner action (inflight and completed) in the active timeline. */ - static boolean isInClosedRange(String timestamp, String startTs, String endTs) { - return HoodieTimeline.compareTimestamps(timestamp, GREATER_THAN_OR_EQUALS, startTs) - && HoodieTimeline.compareTimestamps(timestamp, LESSER_THAN_OR_EQUALS, endTs); - } - - static HoodieInstant getRequestedInstant(final HoodieInstant instant) { - return new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp()); - } - - static HoodieInstant getCleanRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, CLEAN_ACTION, timestamp); - } - - static HoodieInstant getCleanInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, CLEAN_ACTION, timestamp); - } - - static HoodieInstant getCompactionRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, timestamp); - } - - static HoodieInstant getCompactionInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, COMPACTION_ACTION, timestamp); - } - - // Returns Log compaction requested instant - static HoodieInstant getLogCompactionRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, timestamp); - } - - // Returns Log compaction inflight instant - static HoodieInstant getLogCompactionInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, LOG_COMPACTION_ACTION, timestamp); - } - - static HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, REPLACE_COMMIT_ACTION, timestamp); - } - - static HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, REPLACE_COMMIT_ACTION, timestamp); - } - - static HoodieInstant getClusteringCommitRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, CLUSTERING_ACTION, timestamp); - } - - static HoodieInstant getClusteringCommitInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, CLUSTERING_ACTION, timestamp); - } - - static HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { - return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant); - } - - static HoodieInstant getRestoreRequestedInstant(HoodieInstant instant) { - return instant.isRequested() ? instant : HoodieTimeline.getRequestedInstant(instant); - } - - static HoodieInstant getIndexRequestedInstant(final String timestamp) { - return new HoodieInstant(State.REQUESTED, INDEXING_ACTION, timestamp); - } - - static HoodieInstant getIndexInflightInstant(final String timestamp) { - return new HoodieInstant(State.INFLIGHT, INDEXING_ACTION, timestamp); - } + HoodieTimeline getCleanerTimeline(); /** - * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names - * between inflight and completed instants (compaction <=> commit) and (logcompaction <==> deltacommit). - * @param instant Hoodie Instant - * @param metaClient Hoodie metaClient to fetch tableType and fileSystem. - * @return Inflight Hoodie Instant + * Get only the rollback action (inflight and completed) in the active timeline. */ - static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableMetaClient metaClient) { - if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { - if (instant.getAction().equals(COMMIT_ACTION)) { - return new HoodieInstant(true, COMPACTION_ACTION, instant.getTimestamp()); - } else if (instant.getAction().equals(DELTA_COMMIT_ACTION)) { - // 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.getInstantsAsStream() - .filter(hoodieInstant -> hoodieInstant.getTimestamp().equals(instant.getTimestamp()) - && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst()); - if (logCompactionInstant.isPresent()) { - return new HoodieInstant(true, LOG_COMPACTION_ACTION, instant.getTimestamp()); - } - } - } - return new HoodieInstant(true, instant.getAction(), instant.getTimestamp()); - } - - static String makeCommitFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.COMMIT_EXTENSION); - } - - static String makeInflightCommitFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); - } - - static String makeRequestedCommitFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION); - } - - static String makeCleanerFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION); - } - - static String makeRequestedCleanerFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLEAN_EXTENSION); - } - - static String makeInflightCleanerFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION); - } - - static String makeRollbackFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.ROLLBACK_EXTENSION); - } - - static String makeRequestedRollbackFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_ROLLBACK_EXTENSION); - } - - static String makeRequestedRestoreFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_RESTORE_EXTENSION); - } - - static String makeInflightRollbackFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); - } - - static String makeInflightSavePointFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); - } - - static String makeSavePointFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.SAVEPOINT_EXTENSION); - } - - static String makeInflightDeltaFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); - } - - static String makeRequestedDeltaFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION); - } - - static String makeInflightCompactionFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); - } - - static String makeRequestedCompactionFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); - } - - // Log compaction action - static String makeInflightLogCompactionFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_LOG_COMPACTION_EXTENSION); - } - - static String makeRequestedLogCompactionFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_LOG_COMPACTION_EXTENSION); - } - - static String makeRestoreFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION); - } - - static String makeInflightRestoreFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION); - } - - static String makeReplaceFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REPLACE_COMMIT_EXTENSION); - } - - static String makeInflightReplaceFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); - } - - static String makeRequestedReplaceFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION); - } - - static String makeRequestedClusteringFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLUSTERING_COMMIT_EXTENSION); - } - - static String makeInflightClusteringFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLUSTERING_COMMIT_EXTENSION); - } - - static String makeDeltaFileName(String instantTime) { - return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION; - } + HoodieTimeline getRollbackTimeline(); - static String getCommitFromCommitFile(String commitFileName) { - return commitFileName.split("\\.")[0]; - } - - static String makeFileNameAsComplete(String fileName) { - return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, ""); - } - - static String makeFileNameAsInflight(String fileName) { - return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); - } - - static String makeIndexCommitFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INDEX_COMMIT_EXTENSION); - } + /** + * Get only the rollback and restore action (inflight and completed) in the active timeline. + */ + HoodieTimeline getRollbackAndRestoreTimeline(); - static String makeInflightIndexFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.INFLIGHT_INDEX_COMMIT_EXTENSION); - } + /** + * Get only the save point action (inflight and completed) in the active timeline. + */ + HoodieTimeline getSavePointTimeline(); - static String makeRequestedIndexFileName(String instant) { - return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION); - } + /** + * Get only the restore action (inflight and completed) in the active timeline. + */ + HoodieTimeline getRestoreTimeline(); - static String makeSchemaFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION); - } + /** + * Merge this timeline with the given timeline. + */ + HoodieTimeline mergeTimeline(HoodieTimeline timeline); - static String makeInflightSchemaFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION); - } + /** + * Set Instants directly. + */ + void setInstants(List instants); - static String makeRequestSchemaFileName(String instantTime) { - return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION); - } + /** + * Get layout version of this timeline + * @return + */ + TimelineLayoutVersion getTimelineLayoutVersion(); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparator.java new file mode 100644 index 000000000000..bd1b761150ab --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparator.java @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import java.io.Serializable; +import java.util.Comparator; + +public interface InstantComparator extends Serializable { + + /** + * Returns Comparator that only uses action for ordering taking into account equivalent actions. + * @return + */ + Comparator actionOnlyComparator(); + + /** + * Returns comparator that orders primarily based on timestamp and secondary ordering based on action and state. + * @return + */ + Comparator requestedTimeOrderedComparator(); + + /** + * Returns comparaor that orders primarily based on completion time and secondary ordering based on {@link #requestedTimeOrderedComparator()}. + * @return + */ + Comparator completionTimeOrderedComparator(); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparison.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparison.java new file mode 100644 index 000000000000..c9d92b4e82b6 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantComparison.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.common.util.StringUtils; + +import java.util.Objects; +import java.util.function.BiPredicate; + +/** + * Helper methods to compare instants. + **/ +public class InstantComparison { + + public static final BiPredicate EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) == 0; + public static final BiPredicate GREATER_THAN_OR_EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) >= 0; + public static final BiPredicate GREATER_THAN = (commit1, commit2) -> commit1.compareTo(commit2) > 0; + public static final BiPredicate LESSER_THAN_OR_EQUALS = (commit1, commit2) -> commit1.compareTo(commit2) <= 0; + public static final BiPredicate LESSER_THAN = (commit1, commit2) -> commit1.compareTo(commit2) < 0; + + public static boolean compareTimestamps(String commit1, BiPredicate predicateToApply, String commit2) { + return predicateToApply.test(commit1, commit2); + } + + /** + * Returns smaller of the two given timestamps. Returns the non null argument if one of the argument is null. + */ + public static String minTimestamp(String commit1, String commit2) { + if (StringUtils.isNullOrEmpty(commit1)) { + return commit2; + } else if (StringUtils.isNullOrEmpty(commit2)) { + return commit1; + } + return minInstant(commit1, commit2); + } + + /** + * Returns smaller of the two given instants compared by their respective timestamps. + * Returns the non null argument if one of the argument is null. + */ + public static HoodieInstant minTimestampInstant(HoodieInstant instant1, HoodieInstant instant2) { + String commit1 = instant1 != null ? instant1.requestedTime() : null; + String commit2 = instant2 != null ? instant2.requestedTime() : null; + String minTimestamp = minTimestamp(commit1, commit2); + return Objects.equals(minTimestamp, commit1) ? instant1 : instant2; + } + + /** + * Returns the smaller of the given two instants. + */ + public static String minInstant(String instant1, String instant2) { + return compareTimestamps(instant1, LESSER_THAN, instant2) ? instant1 : instant2; + } + + /** + * Returns the greater of the given two instants. + */ + public static String maxInstant(String instant1, String instant2) { + return compareTimestamps(instant1, GREATER_THAN, instant2) ? instant1 : instant2; + } + + /** + * Return true if specified timestamp is in range (startTs, endTs]. + */ + public static boolean isInRange(String timestamp, String startTs, String endTs) { + return compareTimestamps(timestamp, GREATER_THAN, startTs) + && compareTimestamps(timestamp, LESSER_THAN_OR_EQUALS, endTs); + } + + /** + * Return true if specified timestamp is in range [startTs, endTs). + */ + public static boolean isInClosedOpenRange(String timestamp, String startTs, String endTs) { + return compareTimestamps(timestamp, GREATER_THAN_OR_EQUALS, startTs) + && compareTimestamps(timestamp, LESSER_THAN, endTs); + } + + /** + * Return true if specified timestamp is in range [startTs, endTs]. + */ + public static boolean isInClosedRange(String timestamp, String startTs, String endTs) { + return compareTimestamps(timestamp, GREATER_THAN_OR_EQUALS, startTs) + && compareTimestamps(timestamp, LESSER_THAN_OR_EQUALS, endTs); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameGenerator.java new file mode 100644 index 000000000000..ac357b9020ad --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameGenerator.java @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import java.io.Serializable; + +/** + * Factory for generating instants filenames. + */ +public interface InstantFileNameGenerator extends Serializable { + + String makeCommitFileName(String instantTime); + + String makeInflightCommitFileName(String instantTime); + + String makeRequestedCommitFileName(String instantTime); + + String makeCleanerFileName(String instant); + + String makeRequestedCleanerFileName(String instant); + + String makeInflightCleanerFileName(String instant); + + String makeRollbackFileName(String instant); + + String makeRequestedRollbackFileName(String instant); + + String makeRequestedRestoreFileName(String instant); + + String makeInflightRollbackFileName(String instant); + + String makeInflightSavePointFileName(String instantTime); + + String makeSavePointFileName(String instantTime); + + String makeInflightDeltaFileName(String instantTime); + + String makeRequestedDeltaFileName(String instantTime); + + String makeInflightCompactionFileName(String instantTime); + + String makeRequestedCompactionFileName(String instantTime); + + String makeInflightLogCompactionFileName(String instantTime); + + String makeRequestedLogCompactionFileName(String instantTime); + + String makeRestoreFileName(String instant); + + String makeInflightRestoreFileName(String instant); + + String makeReplaceFileName(String instant); + + String makeInflightReplaceFileName(String instant); + + String makeRequestedReplaceFileName(String instant); + + String makeRequestedClusteringFileName(String instant); + + String makeInflightClusteringFileName(String instant); + + String makeDeltaFileName(String instantTime); + + String getCommitFromCommitFile(String commitFileName); + + String makeFileNameAsComplete(String fileName); + + String makeFileNameAsInflight(String fileName); + + String makeIndexCommitFileName(String instant); + + String makeInflightIndexFileName(String instant); + + String makeRequestedIndexFileName(String instant); + + String makeSchemaFileName(String instantTime); + + String makeInflightSchemaFileName(String instantTime); + + String makeRequestSchemaFileName(String instantTime); + + String getFileName(HoodieInstant instant); + + String getFileName(String completionTime, HoodieInstant instant); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameParser.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameParser.java new file mode 100644 index 000000000000..0dd4e23b561d --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantFileNameParser.java @@ -0,0 +1,39 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import java.io.Serializable; + +public interface InstantFileNameParser extends Serializable { + + /** + * Extract timestamp from instant file name + * @param fileName + * @return + * @throws IllegalArgumentException + */ + String extractTimestamp(String fileName) throws IllegalArgumentException; + + /** + * Extract File extension from instant file name + * @param fileName + * @return + */ + String getTimelineFileExtension(String fileName); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantGenerator.java new file mode 100644 index 000000000000..15e1f4e6348c --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/InstantGenerator.java @@ -0,0 +1,69 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.storage.StoragePathInfo; + +import java.io.Serializable; + +import static org.apache.hudi.common.table.timeline.HoodieInstant.State; + +/** + * Factory for generating Instants. + **/ +public interface InstantGenerator extends Serializable { + + HoodieInstant createNewInstant(State state, String action, String timestamp); + + HoodieInstant createNewInstant(State state, String action, String timestamp, String completionTime); + + HoodieInstant createNewInstant(State state, String action, String timestamp, String completionTime, boolean isLegacy); + + HoodieInstant createNewInstant(StoragePathInfo pathInfo); + + HoodieInstant getRequestedInstant(final HoodieInstant instant); + + HoodieInstant getCleanRequestedInstant(final String timestamp); + + HoodieInstant getCleanInflightInstant(final String timestamp); + + HoodieInstant getCompactionRequestedInstant(final String timestamp); + + HoodieInstant getCompactionInflightInstant(final String timestamp); + + HoodieInstant getLogCompactionRequestedInstant(final String timestamp); + + HoodieInstant getLogCompactionInflightInstant(final String timestamp); + + HoodieInstant getReplaceCommitRequestedInstant(final String timestamp); + + HoodieInstant getReplaceCommitInflightInstant(final String timestamp); + + HoodieInstant getClusteringCommitRequestedInstant(final String timestamp); + + HoodieInstant getClusteringCommitInflightInstant(final String timestamp); + + HoodieInstant getRollbackRequestedInstant(HoodieInstant instant); + + HoodieInstant getRestoreRequestedInstant(HoodieInstant instant); + + HoodieInstant getIndexRequestedInstant(final String timestamp); + + HoodieInstant getIndexInflightInstant(final String timestamp); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/MetadataConversionUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/MetadataConversionUtils.java index ed741acf3650..aa397f08cd88 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/MetadataConversionUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/MetadataConversionUtils.java @@ -61,7 +61,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst return createMetaWrapperForEmptyInstant(hoodieInstant); } HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); - archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); + archivedMetaWrapper.setCommitTime(hoodieInstant.requestedTime()); archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); archivedMetaWrapper.setStateTransitionTime(hoodieInstant.getCompletionTime()); switch (hoodieInstant.getAction()) { @@ -75,13 +75,13 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst break; } case HoodieTimeline.COMMIT_ACTION: { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(instantDetails.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(hoodieInstant, instantDetails.get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata)); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } case HoodieTimeline.DELTA_COMMIT_ACTION: { - HoodieCommitMetadata deltaCommitMetadata = HoodieCommitMetadata.fromBytes(instantDetails.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata deltaCommitMetadata = metaClient.getCommitMetadataSerDe().deserialize(hoodieInstant, instantDetails.get(), HoodieCommitMetadata.class); archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(deltaCommitMetadata)); archivedMetaWrapper.setActionType(ActionType.deltacommit.name()); break; @@ -95,7 +95,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst // inflight replacecommit files have the same metadata body as HoodieCommitMetadata // so we could re-use it without further creating an inflight extension. // Or inflight replacecommit files are empty under clustering circumstance - Option inflightCommitMetadata = getInflightCommitMetadata(instantDetails); + Option inflightCommitMetadata = getInflightCommitMetadata(metaClient, hoodieInstant, instantDetails); if (inflightCommitMetadata.isPresent()) { archivedMetaWrapper.setHoodieInflightReplaceMetadata(convertCommitMetadata(inflightCommitMetadata.get())); } @@ -185,7 +185,7 @@ public static HoodieLSMTimelineInstant createLSMTimelineInstant(ActiveAction act public static HoodieArchivedMetaEntry createMetaWrapperForEmptyInstant(HoodieInstant hoodieInstant) { HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); - archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); + archivedMetaWrapper.setCommitTime(hoodieInstant.requestedTime()); archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); archivedMetaWrapper.setStateTransitionTime(hoodieInstant.getCompletionTime()); switch (hoodieInstant.getAction()) { @@ -228,12 +228,13 @@ public static HoodieArchivedMetaEntry createMetaWrapperForEmptyInstant(HoodieIns return archivedMetaWrapper; } - private static Option getInflightCommitMetadata(Option inflightContent) throws IOException { + private static Option getInflightCommitMetadata(HoodieTableMetaClient metaClient, HoodieInstant instant, + Option inflightContent) throws IOException { if (!inflightContent.isPresent() || inflightContent.get().length == 0) { // inflight files can be empty in some certain cases, e.g. when users opt in clustering return Option.empty(); } - return Option.of(HoodieCommitMetadata.fromBytes(inflightContent.get(), HoodieCommitMetadata.class)); + return Option.of(metaClient.getCommitMetadataSerDe().deserialize(instant, inflightContent.get(), HoodieCommitMetadata.class)); } private static Option getRequestedReplaceMetadata(Option requestedContent) throws IOException { 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 a98b71aa5711..502565dd0221 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 @@ -18,6 +18,7 @@ package org.apache.hudi.common.table.timeline; +import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -30,6 +31,9 @@ import java.util.Set; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * A helper class used to diff timeline. */ @@ -40,7 +44,8 @@ public class TimelineDiffHelper { private TimelineDiffHelper() { } - public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline oldTimeline, + public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTableMetaClient metaClient, + HoodieTimeline oldTimeline, HoodieTimeline newTimeline) { HoodieTimeline oldT = oldTimeline.filterCompletedAndCompactionInstants(); @@ -50,8 +55,8 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline Option firstInstantInNewTimeline = newT.firstInstant(); if (lastSeenInstant.isPresent() && firstInstantInNewTimeline.isPresent()) { - if (HoodieTimeline.compareTimestamps(lastSeenInstant.get().getTimestamp(), - HoodieTimeline.LESSER_THAN, firstInstantInNewTimeline.get().getTimestamp())) { + if (compareTimestamps(lastSeenInstant.get().requestedTime(), + LESSER_THAN, firstInstantInNewTimeline.get().requestedTime())) { // The last seen instant is no longer in the timeline. Do not incrementally Sync. return TimelineDiffResult.UNSAFE_SYNC_RESULT; } @@ -60,7 +65,7 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline List newInstants = new ArrayList<>(); // Check If any pending compaction is lost. If so, do not allow incremental timeline sync - List> compactionInstants = getPendingActionTransitions(oldT.filterPendingCompactionTimeline(), + List> compactionInstants = getPendingActionTransitions(metaClient, oldT.filterPendingCompactionTimeline(), newT, HoodieTimeline.COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION); List lostPendingCompactions = compactionInstants.stream() .filter(instantPair -> instantPair.getValue() == null).map(Pair::getKey).collect(Collectors.toList()); @@ -77,7 +82,7 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline newTimeline.getInstantsAsStream().filter(instant -> !oldTimelineInstants.contains(instant)).forEach(newInstants::add); - List> logCompactionInstants = getPendingActionTransitions(oldTimeline.filterPendingLogCompactionTimeline(), + List> logCompactionInstants = getPendingActionTransitions(metaClient, oldTimeline.filterPendingLogCompactionTimeline(), newTimeline, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.LOG_COMPACTION_ACTION); List finishedOrRemovedLogCompactionInstants = logCompactionInstants.stream() .filter(instantPair -> !instantPair.getKey().isCompleted() @@ -91,22 +96,24 @@ public static TimelineDiffResult getNewInstantsForIncrementalSync(HoodieTimeline } } - private static List> getPendingActionTransitions(HoodieTimeline pendingActionTimelineFromOld, - HoodieTimeline newTimeline, - String completedAction, String pendingAction) { + private static List> getPendingActionTransitions(HoodieTableMetaClient metaClient, + HoodieTimeline pendingActionTimelineFromOld, + HoodieTimeline newTimeline, + String completedAction, String pendingAction) { Set newTimelineInstants = newTimeline.getInstantsAsStream().collect(Collectors.toSet()); + InstantGenerator factory = metaClient.getInstantGenerator(); return pendingActionTimelineFromOld.getInstantsAsStream().map(instant -> { if (newTimelineInstants.contains(instant)) { return Pair.of(instant, instant); } else { HoodieInstant completedInstant = - new HoodieInstant(State.COMPLETED, completedAction, instant.getTimestamp()); + factory.createNewInstant(State.COMPLETED, completedAction, instant.requestedTime()); if (newTimelineInstants.contains(completedInstant)) { return Pair.of(instant, completedInstant); } HoodieInstant inflightInstant = - new HoodieInstant(State.INFLIGHT, pendingAction, instant.getTimestamp()); + factory.createNewInstant(State.INFLIGHT, pendingAction, instant.requestedTime()); if (newTimelineInstants.contains(inflightInstant)) { return Pair.of(instant, inflightInstant); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineFactory.java new file mode 100644 index 000000000000..f1e101b61fef --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineFactory.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.Option; + +import java.io.Serializable; +import java.util.function.Function; +import java.util.stream.Stream; + +public abstract class TimelineFactory implements Serializable { + + public abstract HoodieTimeline createDefaultTimeline(Stream instants, + Function> details); + + public abstract HoodieActiveTimeline createActiveTimeline(); + + public abstract HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient); + + public abstract HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient, String startTs); + + public abstract ArchivedTimelineLoader createArchivedTimelineLoader(); + + public abstract HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient); + + public abstract HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter); + + public abstract CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient); + + public abstract CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient, String eagerInstant); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java index d8616fc40972..c31522962352 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineLayout.java @@ -18,12 +18,24 @@ package org.apache.hudi.common.table.timeline; +import org.apache.hudi.common.table.timeline.versioning.v1.CommitMetadataSerDeV1; +import org.apache.hudi.common.table.timeline.versioning.v1.InstantComparatorV1; +import org.apache.hudi.common.table.timeline.versioning.v1.InstantGeneratorV1; +import org.apache.hudi.common.table.timeline.versioning.v1.InstantFileNameGeneratorV1; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.timeline.versioning.v1.TimelineV1Factory; +import org.apache.hudi.common.table.timeline.versioning.v2.CommitMetadataSerDeV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantGeneratorV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameGeneratorV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameParserV2; +import org.apache.hudi.common.table.timeline.versioning.v2.TimelineV2Factory; import org.apache.hudi.common.util.collection.Pair; import java.io.Serializable; import java.util.HashMap; import java.util.Map; +import java.util.function.Function; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -37,41 +49,142 @@ public abstract class TimelineLayout implements Serializable { static { LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_0), new TimelineLayoutV0()); LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_1), new TimelineLayoutV1()); + LAYOUT_MAP.put(new TimelineLayoutVersion(TimelineLayoutVersion.VERSION_2), new TimelineLayoutV2()); } - public static TimelineLayout getLayout(TimelineLayoutVersion version) { + public static TimelineLayout fromVersion(TimelineLayoutVersion version) { return LAYOUT_MAP.get(version); } public abstract Stream filterHoodieInstants(Stream instantStream); + public abstract InstantGenerator getInstantGenerator(); + + public abstract InstantFileNameGenerator getInstantFileNameGenerator(); + + public abstract TimelineFactory getTimelineFactory(); + + public abstract InstantComparator getInstantComparator(); + + public abstract InstantFileNameParser getInstantFileNameParser(); + + public abstract CommitMetadataSerDe getCommitMetadataSerDe(); + /** * Table Layout where state transitions are managed by renaming files. */ private static class TimelineLayoutV0 extends TimelineLayout { + private final InstantGenerator instantGenerator = new InstantGeneratorV1(); + private final InstantFileNameGenerator instantFileNameGenerator = new InstantFileNameGeneratorV1(); + private final TimelineFactory timelineFactory = new TimelineV1Factory(this); + private final InstantComparator instantComparator = new InstantComparatorV1(); + private final InstantFileNameParser fileNameParser = new InstantFileNameParserV2(); + @Override public Stream filterHoodieInstants(Stream instantStream) { return instantStream; } + + @Override + public InstantGenerator getInstantGenerator() { + return instantGenerator; + } + + @Override + public InstantFileNameGenerator getInstantFileNameGenerator() { + return instantFileNameGenerator; + } + + @Override + public TimelineFactory getTimelineFactory() { + return timelineFactory; + } + + @Override + public InstantComparator getInstantComparator() { + return instantComparator; + } + + @Override + public InstantFileNameParser getInstantFileNameParser() { + return fileNameParser; + } + + @Override + public CommitMetadataSerDe getCommitMetadataSerDe() { + return new CommitMetadataSerDeV1(); + } + } + + private static Stream filterHoodieInstantsByLatestState(Stream instantStream, + Function actionMapper) { + return instantStream.collect(Collectors.groupingBy(instant -> Pair.of(instant.requestedTime(), + actionMapper.apply(instant.getAction())))).values().stream() + .map(hoodieInstants -> hoodieInstants.stream().reduce((x, y) -> { + // Pick the one with the highest state + if (x.getState().compareTo(y.getState()) >= 0) { + return x; + } + return y; + }).get()); } /** * Table Layout where state transitions are managed by creating new files. */ - private static class TimelineLayoutV1 extends TimelineLayout { + private static class TimelineLayoutV1 extends TimelineLayoutV0 { @Override public Stream filterHoodieInstants(Stream instantStream) { - return instantStream.collect(Collectors.groupingBy(instant -> Pair.of(instant.getTimestamp(), - HoodieInstant.getComparableAction(instant.getAction())))).values().stream() - .map(hoodieInstants -> hoodieInstants.stream().reduce((x, y) -> { - // Pick the one with the highest state - if (x.getState().compareTo(y.getState()) >= 0) { - return x; - } - return y; - }).get()); + return TimelineLayout.filterHoodieInstantsByLatestState(instantStream, InstantComparatorV1::getComparableAction); + } + } + + /** + * Timeline corresponding to Hudi 1.x + */ + private static class TimelineLayoutV2 extends TimelineLayout { + + private final InstantGenerator instantGenerator = new InstantGeneratorV2(); + private final InstantFileNameGenerator instantFileNameGenerator = new InstantFileNameGeneratorV2(); + private final TimelineFactory timelineFactory = new TimelineV2Factory(this); + private final InstantComparator instantComparator = new InstantComparatorV2(); + private final InstantFileNameParser fileNameParser = new InstantFileNameParserV2(); + + @Override + public Stream filterHoodieInstants(Stream instantStream) { + return TimelineLayout.filterHoodieInstantsByLatestState(instantStream, InstantComparatorV2::getComparableAction); + } + + @Override + public InstantGenerator getInstantGenerator() { + return instantGenerator; + } + + @Override + public InstantFileNameGenerator getInstantFileNameGenerator() { + return instantFileNameGenerator; + } + + @Override + public TimelineFactory getTimelineFactory() { + return timelineFactory; + } + + @Override + public InstantComparator getInstantComparator() { + return instantComparator; + } + + @Override + public InstantFileNameParser getInstantFileNameParser() { + return fileNameParser; + } + + @Override + public CommitMetadataSerDe getCommitMetadataSerDe() { + return new CommitMetadataSerDeV2(); } } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index 77f6c172259e..5b79ce11b791 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -69,9 +69,9 @@ public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTi List instants, Map> instantToRollbackMetadata) { return new HoodieRestoreMetadata(startRestoreTime, durationInMs, - instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), + instants.stream().map(HoodieInstant::requestedTime).collect(Collectors.toList()), Collections.unmodifiableMap(instantToRollbackMetadata), DEFAULT_VERSION, - instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList())); + instants.stream().map(instant -> new HoodieInstantInfo(instant.requestedTime(), instant.getAction())).collect(Collectors.toList())); } public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, Option durationInMs, @@ -88,9 +88,9 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac } return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L), totalDeleted, - instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), + instants.stream().map(HoodieInstant::requestedTime).collect(Collectors.toList()), Collections.unmodifiableMap(partitionMetadataBuilder), DEFAULT_VERSION, - instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList())); + instants.stream().map(instant -> new HoodieInstantInfo(instant.requestedTime(), instant.getAction())).collect(Collectors.toList())); } public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment, @@ -148,19 +148,12 @@ public static Option serializeIndexCommitMetadata(HoodieIndexCommitMetad return serializeAvroMetadata(indexCommitMetadata, HoodieIndexCommitMetadata.class); } - public static Option serializeCommitMetadata(org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata) throws IOException { + public static Option serializeCommitMetadata(CommitMetadataSerDe commitMetadataSerDe, + org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata) throws IOException { if (commitMetadata instanceof org.apache.hudi.common.model.HoodieReplaceCommitMetadata) { - return serializeReplaceCommitMetadata(MetadataConversionUtils.convertCommitMetadata(commitMetadata)); + return serializeAvroMetadata(MetadataConversionUtils.convertCommitMetadata(commitMetadata), HoodieReplaceCommitMetadata.class); } - return serializeCommitMetadata(MetadataConversionUtils.convertCommitMetadata(commitMetadata)); - } - - private static Option serializeCommitMetadata(HoodieCommitMetadata commitMetadata) throws IOException { - return serializeAvroMetadata(commitMetadata, HoodieCommitMetadata.class); - } - - private static Option serializeReplaceCommitMetadata(HoodieReplaceCommitMetadata commitMetadata) throws IOException { - return serializeAvroMetadata(commitMetadata, HoodieReplaceCommitMetadata.class); + return commitMetadataSerDe.serialize(commitMetadata); } public static Option serializeAvroMetadata(T metadata, Class clazz) 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 3d8d31193ef9..42a92a453b64 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 @@ -23,6 +23,7 @@ import org.apache.hudi.common.config.HoodieCommonConfig; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.util.CleanerUtils; @@ -35,14 +36,18 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.exception.HoodieTimeTravelException; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; +import java.text.ParseException; import java.util.AbstractMap; import java.util.Collection; import java.util.Collections; +import java.util.Date; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -53,14 +58,16 @@ import static org.apache.hudi.common.config.HoodieCommonConfig.INCREMENTAL_READ_HANDLE_HOLLOW_COMMIT; import static org.apache.hudi.common.table.timeline.HoodieTimeline.CLUSTERING_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVEPOINT_ACTION; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.compareTimestamps; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * TimelineUtils provides a common way to query incremental meta-data changes for a hoodie table. @@ -70,6 +77,13 @@ * 2) Incremental reads - InputFormats can use this API to query */ public class TimelineUtils { + public static final Set NOT_PARSABLE_TIMESTAMPS = new HashSet(3) { + { + add(HoodieTimeline.INIT_INSTANT_TS); + add(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS); + add(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS); + } + }; private static final Logger LOG = LoggerFactory.getLogger(TimelineUtils.class); /** @@ -103,7 +117,7 @@ public static List getDroppedPartitions(HoodieTableMetaClient metaClient }) .filter(pair -> isDeletePartition(pair.getRight().getOperationType())) .flatMap(pair -> pair.getRight().getPartitionToReplaceFileIds().keySet().stream() - .map(partition -> new AbstractMap.SimpleEntry<>(partition, pair.getLeft().getTimestamp())) + .map(partition -> new AbstractMap.SimpleEntry<>(partition, pair.getLeft().requestedTime())) ).collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, (existing, replace) -> replace)); // cleaner could delete a partition when there are no active filegroups in the partition HoodieTimeline cleanerTimeline = metaClient.getActiveTimeline().getCleanerTimeline().filterCompletedInstants(); @@ -113,7 +127,7 @@ public static List getDroppedPartitions(HoodieTableMetaClient metaClient HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(cleanerTimeline.getInstantDetails(instant).get()); cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { if (partitionMetadata.getIsPartitionDeleted()) { - partitionToLatestDeleteTimestamp.put(partition, instant.getTimestamp()); + partitionToLatestDeleteTimestamp.put(partition, instant.requestedTime()); } }); } catch (IOException e) { @@ -129,12 +143,12 @@ public static List getDroppedPartitions(HoodieTableMetaClient metaClient .reduce((left, right) -> compareTimestamps(left, LESSER_THAN, right) ? left : right) .get(); Map partitionToLatestWriteTimestamp = completedTimeline.getInstantsAsStream() - .filter(instant -> compareTimestamps(instant.getTimestamp(), GREATER_THAN_OR_EQUALS, earliestDeleteTimestamp)) + .filter(instant -> compareTimestamps(instant.requestedTime(), GREATER_THAN_OR_EQUALS, earliestDeleteTimestamp)) .flatMap(instant -> { try { HoodieCommitMetadata commitMetadata = getCommitMetadata(instant, completedTimeline); return commitMetadata.getWritePartitionPaths().stream() - .map(partition -> new AbstractMap.SimpleEntry<>(partition, instant.getTimestamp())); + .map(partition -> new AbstractMap.SimpleEntry<>(partition, instant.requestedTime())); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions writes at " + instant, e); } @@ -155,7 +169,8 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { case COMMIT_ACTION: case DELTA_COMMIT_ACTION: try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class); + CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); + HoodieCommitMetadata commitMetadata = metadataSerDe.deserialize(s, timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class); return commitMetadata.getPartitionToWriteStats().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions written at " + s, e); @@ -235,13 +250,13 @@ public static Option getExtraMetadataFromLatestIncludeClustering(HoodieT */ public static Map> getAllExtraMetadataForKey(HoodieTableMetaClient metaClient, String extraMetadataKey) { return metaClient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants().collect(Collectors.toMap( - HoodieInstant::getTimestamp, instant -> getMetadataValue(metaClient, extraMetadataKey, instant))); + HoodieInstant::requestedTime, instant -> getMetadataValue(metaClient, extraMetadataKey, instant))); } private static Option getMetadataValue(HoodieTableMetaClient metaClient, String extraMetadataKey, HoodieInstant instant) { try { LOG.info("reading checkpoint info for:" + instant + " key: " + extraMetadataKey); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, metaClient.getCommitsTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); return Option.ofNullable(commitMetadata.getExtraMetadata().get(extraMetadataKey)); @@ -267,7 +282,7 @@ public static boolean isClusteringCommit(HoodieTableMetaClient metaClient, Hoodi } } - public static HoodieDefaultTimeline getTimeline(HoodieTableMetaClient metaClient, boolean includeArchivedTimeline) { + public static HoodieTimeline getTimeline(HoodieTableMetaClient metaClient, boolean includeArchivedTimeline) { HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); if (includeArchivedTimeline) { HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); @@ -286,19 +301,19 @@ public static HoodieDefaultTimeline getTimeline(HoodieTableMetaClient metaClient */ public static HoodieTimeline getCommitsTimelineAfter( HoodieTableMetaClient metaClient, String exclusiveStartInstantTime, Option lastMaxCompletionTime) { - HoodieDefaultTimeline writeTimeline = metaClient.getActiveTimeline().getWriteTimeline(); + HoodieTimeline writeTimeline = metaClient.getActiveTimeline().getWriteTimeline(); - HoodieDefaultTimeline timeline = writeTimeline.isBeforeTimelineStarts(exclusiveStartInstantTime) + HoodieTimeline timeline = writeTimeline.isBeforeTimelineStarts(exclusiveStartInstantTime) ? metaClient.getArchivedTimeline(exclusiveStartInstantTime).mergeTimeline(writeTimeline) : writeTimeline; - HoodieDefaultTimeline timelineSinceLastSync = (HoodieDefaultTimeline) timeline.getCommitsTimeline() - .findInstantsAfter(exclusiveStartInstantTime, Integer.MAX_VALUE); + HoodieTimeline timelineSinceLastSync = (HoodieTimeline) (((HoodieTimeline) timeline).getCommitsTimeline() + .findInstantsAfter(exclusiveStartInstantTime, Integer.MAX_VALUE)); if (lastMaxCompletionTime.isPresent()) { // Get 'hollow' instants that have less instant time than exclusiveStartInstantTime but with greater commit completion time - HoodieDefaultTimeline hollowInstantsTimeline = (HoodieDefaultTimeline) timeline.getCommitsTimeline() - .filter(s -> compareTimestamps(s.getTimestamp(), LESSER_THAN, exclusiveStartInstantTime)) + HoodieTimeline hollowInstantsTimeline = ((HoodieTimeline) timeline).getCommitsTimeline() + .filter(s -> compareTimestamps(s.requestedTime(), LESSER_THAN, exclusiveStartInstantTime)) .filter(s -> compareTimestamps(s.getCompletionTime(), GREATER_THAN, lastMaxCompletionTime.get())); if (!hollowInstantsTimeline.empty()) { return timelineSinceLastSync.mergeTimeline(hollowInstantsTimeline); @@ -322,7 +337,8 @@ public static HoodieCommitMetadata getCommitMetadata( if (instant.getAction().equals(REPLACE_COMMIT_ACTION) || instant.getAction().equals(CLUSTERING_ACTION)) { return HoodieReplaceCommitMetadata.fromBytes(data, HoodieReplaceCommitMetadata.class); } else { - return HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); + return metadataSerDe.deserialize(instant, data, HoodieCommitMetadata.class); } } @@ -377,11 +393,11 @@ public static Option getEarliestInstantForMetadataArchival( public static void validateTimestampAsOf(HoodieTableMetaClient metaClient, String timestampAsOf) { Option firstIncompleteCommit = metaClient.getCommitsTimeline() .filterInflightsAndRequested() - .filter(instant -> !ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), instant)) + .filter(instant -> !ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), instant, metaClient.getInstantGenerator())) .firstInstant(); if (firstIncompleteCommit.isPresent()) { - String incompleteCommitTime = firstIncompleteCommit.get().getTimestamp(); + String incompleteCommitTime = firstIncompleteCommit.get().requestedTime(); if (compareTimestamps(timestampAsOf, GREATER_THAN_OR_EQUALS, incompleteCommitTime)) { throw new HoodieTimeTravelException(String.format( "Time travel's timestamp '%s' must be earlier than the first incomplete commit timestamp '%s'.", @@ -397,7 +413,7 @@ public static void validateTimestampAsOf(HoodieTableMetaClient metaClient, Strin HoodieCleanMetadata cleanMetadata = CleanerUtils.getCleanerMetadata(metaClient, latestCleanOpt.get()); String earliestCommitToRetain = cleanMetadata.getEarliestCommitToRetain(); if (!StringUtils.isNullOrEmpty(earliestCommitToRetain)) { - ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(earliestCommitToRetain, LESSER_THAN_OR_EQUALS, timestampAsOf), + ValidationUtils.checkArgument(compareTimestamps(earliestCommitToRetain, LESSER_THAN_OR_EQUALS, timestampAsOf), "Cleaner cleaned up the timestamp of interest. Please ensure sufficient commits are retained with cleaner " + "for Timestamp as of query to work"); } else { @@ -405,7 +421,7 @@ public static void validateTimestampAsOf(HoodieTableMetaClient metaClient, Strin // so, lets check if timestamp of interest is archived based on first entry in active timeline Option firstCompletedInstant = metaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().firstInstant(); if (firstCompletedInstant.isPresent()) { - ValidationUtils.checkArgument(HoodieTimeline.compareTimestamps(firstCompletedInstant.get().getTimestamp(), LESSER_THAN_OR_EQUALS, timestampAsOf), + ValidationUtils.checkArgument(compareTimestamps(firstCompletedInstant.get().requestedTime(), LESSER_THAN_OR_EQUALS, timestampAsOf), "Please ensure sufficient commits are retained (uncleaned and un-archived) for timestamp as of query to work."); } } @@ -428,17 +444,17 @@ public static HoodieTimeline handleHollowCommitIfNeeded(HoodieTimeline completed Option firstIncompleteCommit = metaClient.getCommitsTimeline() .filterInflightsAndRequested() - .filter(instant -> !ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), instant)) + .filter(instant -> !ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), instant, metaClient.getInstantGenerator())) .firstInstant(); boolean noHollowCommit = firstIncompleteCommit - .map(i -> completedCommitTimeline.findInstantsAfter(i.getTimestamp()).empty()) + .map(i -> completedCommitTimeline.findInstantsAfter(i.requestedTime()).empty()) .orElse(true); if (noHollowCommit) { return completedCommitTimeline; } - String hollowCommitTimestamp = firstIncompleteCommit.get().getTimestamp(); + String hollowCommitTimestamp = firstIncompleteCommit.get().requestedTime(); switch (handlingMode) { case FAIL: throw new HoodieException(String.format( @@ -454,6 +470,120 @@ public static HoodieTimeline handleHollowCommitIfNeeded(HoodieTimeline completed } } + /** + * Parse the timestamp of an Instant and return a {@code Date}. + * Throw ParseException if timestamp is not valid format as + * {@link HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. + * + * @param timestamp a timestamp String which follow pattern as + * {@link HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. + * @return Date of instant timestamp + */ + public static Date parseDateFromInstantTime(String timestamp) throws ParseException { + return HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp); + } + + /** + * The same parsing method as above, but this method will mute ParseException. + * If the given timestamp is invalid, returns {@code Option.empty}. + * Or a corresponding Date value if these timestamp strings are provided + * {@link HoodieTimeline#INIT_INSTANT_TS}, + * {@link HoodieTimeline#METADATA_BOOTSTRAP_INSTANT_TS}, + * {@link HoodieTimeline#FULL_BOOTSTRAP_INSTANT_TS}. + * This method is useful when parsing timestamp for metrics + * + * @param timestamp a timestamp String which follow pattern as + * {@link HoodieInstantTimeGenerator#SECS_INSTANT_TIMESTAMP_FORMAT}. + * @return {@code Option} of instant timestamp, {@code Option.empty} if invalid timestamp + */ + public static Option parseDateFromInstantTimeSafely(String timestamp) { + Option parsedDate; + try { + parsedDate = Option.of(HoodieInstantTimeGenerator.parseDateFromInstantTime(timestamp)); + } catch (ParseException e) { + if (NOT_PARSABLE_TIMESTAMPS.contains(timestamp)) { + parsedDate = Option.of(new Date(Integer.parseInt(timestamp))); + } else { + LOG.warn("Failed to parse timestamp {}: {}", timestamp, e.getMessage()); + parsedDate = Option.empty(); + } + } + return parsedDate; + } + + /** + * Format the Date to a String representing the timestamp of a Hoodie Instant. + */ + public static String formatDate(Date timestamp) { + return HoodieInstantTimeGenerator.formatDate(timestamp); + } + + /** + * Returns next instant time in the correct format. + * Ensures each instant time is at least 1 millisecond apart since we create instant times at millisecond granularity. + * + * @param shouldLock whether the lock should be enabled to get the instant time. + * @param timeGenerator TimeGenerator used to generate the instant time. + */ + public static String generateInstantTime(boolean shouldLock, TimeGenerator timeGenerator) { + return generateInstantTime(shouldLock, timeGenerator, 0L); + } + + /** + * Returns next instant time in the correct format. + * Ensures each instant time is at least 1 millisecond apart since we create instant times at millisecond granularity. + * + * @param shouldLock whether the lock should be enabled to get the instant time. + * @param timeGenerator TimeGenerator used to generate the instant time. + * @param milliseconds Milliseconds to add to current time while generating the new instant time + */ + public static String generateInstantTime(boolean shouldLock, TimeGenerator timeGenerator, long milliseconds) { + return HoodieInstantTimeGenerator.createNewInstantTime(shouldLock, timeGenerator, milliseconds); + } + + /** + * Delete Instant file from storage + * @param storage Hoodie Storage. + * @param metaPath Path. + * @param instant instant to delete. + * @param factory Factory to generate file name. + */ + public static void deleteInstantFile(HoodieStorage storage, StoragePath metaPath, HoodieInstant instant, InstantFileNameGenerator factory) { + String filePath = factory.getFileName(instant); + try { + storage.deleteFile(new StoragePath(metaPath, filePath)); + } catch (IOException e) { + throw new HoodieIOException("Could not delete instant file" + filePath, e); + } + } + + /** + * Returns the inflight instant corresponding to the instant being passed. Takes care of changes in action names + * between inflight and completed instants (compaction <=> commit) and (logcompaction <==> deltacommit). + * @param instant Hoodie Instant + * @param metaClient Hoodie metaClient to fetch tableType and fileSystem. + * @return Inflight Hoodie Instant + */ + public static HoodieInstant getInflightInstant(final HoodieInstant instant, final HoodieTableMetaClient metaClient) { + InstantGenerator factory = metaClient.getInstantGenerator(); + if (metaClient.getTableType() == HoodieTableType.MERGE_ON_READ) { + if (instant.getAction().equals(COMMIT_ACTION)) { + return factory.createNewInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, instant.requestedTime()); + } else if (instant.getAction().equals(DELTA_COMMIT_ACTION)) { + // Deltacommit is used by both ingestion and logcompaction. + // So, distinguish both of them check for the inflight file being present. + HoodieActiveTimeline rawActiveTimeline = metaClient.getTimelineLayout().getTimelineFactory().createActiveTimeline(metaClient, false); + Option logCompactionInstant = Option.fromJavaOptional(rawActiveTimeline.getInstantsAsStream() + .filter(hoodieInstant -> hoodieInstant.requestedTime().equals(instant.requestedTime()) + && LOG_COMPACTION_ACTION.equals(hoodieInstant.getAction())).findFirst()); + if (logCompactionInstant.isPresent()) { + return factory.createNewInstant(HoodieInstant.State.INFLIGHT, LOG_COMPACTION_ACTION, instant.requestedTime()); + } + } + } + return factory.createNewInstant(HoodieInstant.State.INFLIGHT, instant.getAction(), instant.requestedTime()); + } + public enum HollowCommitHandling { FAIL, BLOCK, USE_TRANSITION_TIME } @@ -463,7 +593,7 @@ public enum HollowCommitHandling { */ public static HoodieTimeline concatTimeline(HoodieTimeline timeline1, HoodieTimeline timeline2, HoodieTableMetaClient metaClient) { - return new HoodieDefaultTimeline(Stream.concat(timeline1.getInstantsAsStream(), timeline2.getInstantsAsStream()).sorted(), + return metaClient.getTimelineLayout().getTimelineFactory().createDefaultTimeline(Stream.concat(timeline1.getInstantsAsStream(), timeline2.getInstantsAsStream()).sorted(), instant -> metaClient.getActiveTimeline().getInstantDetails(instant)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/ClusteringOpDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/ClusteringOpDTO.java index ba69dd919228..2e7fbdbf7de0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/ClusteringOpDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/ClusteringOpDTO.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.collection.Pair; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; @@ -52,12 +53,12 @@ public static ClusteringOpDTO fromClusteringOp(HoodieFileGroupId fileGroupId, Ho dto.partitionPath = fileGroupId.getPartitionPath(); dto.instantAction = instant.getAction(); dto.instantState = instant.getState().name(); - dto.instantTime = instant.getTimestamp(); + dto.instantTime = instant.requestedTime(); return dto; } - public static Pair toClusteringOperation(ClusteringOpDTO dto) { + public static Pair toClusteringOperation(ClusteringOpDTO dto, InstantGenerator factory) { return Pair.of(new HoodieFileGroupId(dto.partitionPath, dto.fileId), - new HoodieInstant(HoodieInstant.State.valueOf(dto.instantState), dto.instantAction, dto.instantTime)); + factory.createNewInstant(HoodieInstant.State.valueOf(dto.instantState), dto.instantAction, dto.instantTime)); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java index 39a16488ebfd..006daf88a538 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/dto/InstantDTO.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.table.timeline.InstantGenerator; /** * The data transfer object of instant. @@ -43,16 +44,16 @@ public static InstantDTO fromInstant(HoodieInstant instant) { InstantDTO dto = new InstantDTO(); dto.action = instant.getAction(); - dto.timestamp = instant.getTimestamp(); + dto.timestamp = instant.requestedTime(); dto.state = instant.getState().toString(); return dto; } - public static HoodieInstant toInstant(InstantDTO dto) { + public static HoodieInstant toInstant(InstantDTO dto, InstantGenerator factory) { if (null == dto) { return null; } - return new HoodieInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp); + return factory.createNewInstant(HoodieInstant.State.valueOf(dto.state), dto.action, dto.timestamp); } } 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 740dcea5bfe7..543d241a5739 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 @@ -19,11 +19,12 @@ package org.apache.hudi.common.table.timeline.dto; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import com.fasterxml.jackson.annotation.JsonIgnoreProperties; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import java.util.List; import java.util.stream.Collectors; @@ -44,8 +45,10 @@ public static TimelineDTO fromTimeline(HoodieTimeline timeline) { } public static HoodieTimeline toTimeline(TimelineDTO dto, HoodieTableMetaClient metaClient) { + InstantGenerator instantGenerator = metaClient.getInstantGenerator(); + TimelineFactory factory = metaClient.getTimelineLayout().getTimelineFactory(); // TODO: For Now, we will assume, only active-timeline will be transferred. - return new HoodieDefaultTimeline(dto.instants.stream().map(InstantDTO::toInstant), + return factory.createDefaultTimeline(dto.instants.stream().map(d -> InstantDTO.toInstant(d, instantGenerator)), metaClient.getActiveTimeline()::getInstantDetails); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultCommitMetadataSerDe.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultCommitMetadataSerDe.java new file mode 100644 index 000000000000..40a68e237533 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultCommitMetadataSerDe.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning; + +import org.apache.hudi.common.table.timeline.versioning.v2.CommitMetadataSerDeV2; + +public class DefaultCommitMetadataSerDe extends CommitMetadataSerDeV2 { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameGenerator.java new file mode 100644 index 000000000000..0e948cf5be47 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameGenerator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning; + +import org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameGeneratorV2; + +public class DefaultInstantFileNameGenerator extends InstantFileNameGeneratorV2 { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameParser.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameParser.java new file mode 100644 index 000000000000..7f12417182cc --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantFileNameParser.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning; + +import org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameParserV2; + +public class DefaultInstantFileNameParser extends InstantFileNameParserV2 { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantGenerator.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantGenerator.java new file mode 100644 index 000000000000..17ecabab0db0 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultInstantGenerator.java @@ -0,0 +1,24 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning; + +import org.apache.hudi.common.table.timeline.versioning.v2.InstantGeneratorV2; + +public class DefaultInstantGenerator extends InstantGeneratorV2 { +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultTimelineFactory.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultTimelineFactory.java new file mode 100644 index 000000000000..c4d897d67e28 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/DefaultTimelineFactory.java @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning; + +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.versioning.v2.TimelineV2Factory; + +public class DefaultTimelineFactory extends TimelineV2Factory { + + public DefaultTimelineFactory() { + super(TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION)); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/TimelineLayoutVersion.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/TimelineLayoutVersion.java index 0ed83ab59c05..feceb81e33b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/TimelineLayoutVersion.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/TimelineLayoutVersion.java @@ -29,10 +29,15 @@ public class TimelineLayoutVersion implements Serializable, Comparable { public static final Integer VERSION_0 = 0; // pre 0.5.1 version format - public static final Integer VERSION_1 = 1; // current version with no renames + public static final Integer VERSION_1 = 1; // version with no renames for 0.x + public static final Integer VERSION_2 = 2; // version with completion time in instant filenames and other changes specific to 1.x + + public static final Integer CURR_VERSION = VERSION_2; + public static final TimelineLayoutVersion LAYOUT_VERSION_0 = new TimelineLayoutVersion(VERSION_0); + public static final TimelineLayoutVersion LAYOUT_VERSION_1 = new TimelineLayoutVersion(VERSION_1); + public static final TimelineLayoutVersion LAYOUT_VERSION_2 = new TimelineLayoutVersion(VERSION_2); + public static final TimelineLayoutVersion CURR_LAYOUT_VERSION = LAYOUT_VERSION_2; - public static final Integer CURR_VERSION = VERSION_1; - public static final TimelineLayoutVersion CURR_LAYOUT_VERSION = new TimelineLayoutVersion(CURR_VERSION); private final Integer version; diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/common/InstantComparators.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/common/InstantComparators.java new file mode 100644 index 000000000000..518b5bc93821 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/common/InstantComparators.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.common; + +import org.apache.hudi.common.table.timeline.HoodieInstant; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.Map; + +/** + * Comparators for HoodieInstant that are also serializable. + * java.util.Comparators are not serializable. + */ +public class InstantComparators { + + public static class ActionComparator implements Serializable, Comparator { + private final Map comparableActions; + + public ActionComparator(Map comparableActions) { + this.comparableActions = comparableActions; + } + + @Override + public int compare(HoodieInstant instant1, HoodieInstant instant2) { + String action1 = getComparableAction(instant1.getAction()); + String action2 = getComparableAction(instant2.getAction()); + return action1.compareTo(action2); + } + + private String getComparableAction(String action) { + return comparableActions.getOrDefault(action, action); + } + + } + + public static class RequestedTimeBasedComparator implements Serializable, Comparator { + private final ActionComparator actionComparator; + + public RequestedTimeBasedComparator(Map comparableActions) { + this.actionComparator = new ActionComparator(comparableActions); + } + + @Override + public int compare(HoodieInstant instant1, HoodieInstant instant2) { + int res = instant1.requestedTime().compareTo(instant2.requestedTime()); + if (res == 0) { + res = actionComparator.compare(instant1, instant2); + if (res == 0) { + res = instant1.getState().compareTo(instant2.getState()); + } + } + return res; + } + } + + public static class CompletionTimeBasedComparator implements Serializable, Comparator { + private final RequestedTimeBasedComparator timestampBasedComparator; + + public CompletionTimeBasedComparator(Map comparableActions) { + this.timestampBasedComparator = new RequestedTimeBasedComparator(comparableActions); + } + + @Override + public int compare(HoodieInstant instant1, HoodieInstant instant2) { + int res = instant1.getCompletionTime().compareTo(instant2.getCompletionTime()); + if (res == 0) { + res = timestampBasedComparator.compare(instant1, instant2); + } + return res; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java new file mode 100644 index 000000000000..cc58d0159c14 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ActiveTimelineV1.java @@ -0,0 +1,702 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Stream; + +public class ActiveTimelineV1 extends BaseTimelineV1 implements HoodieActiveTimeline { + + public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( + COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, + DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, + SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, + INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, + REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + INFLIGHT_LOG_COMPACTION_EXTENSION, REQUESTED_LOG_COMPACTION_EXTENSION, + ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, + REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION, + REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION)); + + private static final Logger LOG = LoggerFactory.getLogger(HoodieActiveTimeline.class); + protected HoodieTableMetaClient metaClient; + private final InstantFileNameGenerator instantFileNameGenerator = new InstantFileNameGeneratorV1(); + + protected ActiveTimelineV1(HoodieTableMetaClient metaClient, Set includedExtensions, + boolean applyLayoutFilters) { + // Filter all the filter in the metapath and include only the extensions passed and + // convert them into HoodieInstant + try { + this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(includedExtensions, applyLayoutFilters)); + } catch (IOException e) { + throw new HoodieIOException("Failed to scan metadata", e); + } + this.metaClient = metaClient; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + LOG.info("Loaded instants upto : " + lastInstant()); + } + + public ActiveTimelineV1(HoodieTableMetaClient metaClient) { + this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), true); + } + + public ActiveTimelineV1(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { + this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), applyLayoutFilter); + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + @Deprecated + public ActiveTimelineV1() { + } + + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + @Deprecated + private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } + + @Override + public Set getValidExtensionsInActiveTimeline() { + return Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); + } + + @Override + public void createCompleteInstant(HoodieInstant instant) { + LOG.info("Creating a new complete instant " + instant); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), Option.empty(), false); + } + + @Override + public void createNewInstant(HoodieInstant instant) { + LOG.info("Creating a new instant " + instant); + // Create the in-flight file + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), Option.empty(), false); + } + + @Override + public void createRequestedCommitWithReplaceMetadata(String instantTime, String actionType) { + try { + HoodieInstant instant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, actionType, instantTime); + LOG.info("Creating a new instant " + instant); + // Create the request replace file + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), + TimelineMetadataUtils.serializeRequestedReplaceMetadata(new HoodieRequestedReplaceMetadata()), false); + } catch (IOException e) { + throw new HoodieIOException("Error create requested replace commit ", e); + } + } + + @Override + public void saveAsComplete(HoodieInstant instant, Option data) { + LOG.info("Marking instant complete " + instant); + ValidationUtils.checkArgument(instant.isInflight(), + "Could not mark an already completed instant as complete again " + instant); + transitionState(instant, instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, instant.getAction(), instant.requestedTime()), data); + LOG.info("Completed " + instant); + } + + @Override + public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option data) { + saveAsComplete(instant, data); + } + + @Override + public HoodieInstant revertToInflight(HoodieInstant instant) { + LOG.info("Reverting instant to inflight " + instant); + HoodieInstant inflight = TimelineUtils.getInflightInstant(instant, metaClient); + revertCompleteToInflight(instant, inflight); + LOG.info("Reverted " + instant + " to inflight " + inflight); + return inflight; + } + + @Override + public void deleteInflight(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isInflight()); + deleteInstantFile(instant); + } + + @Override + public void deletePending(HoodieInstant instant) { + ValidationUtils.checkArgument(!instant.isCompleted()); + deleteInstantFile(instant); + } + + @Override + public void deleteCompletedRollback(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isCompleted()); + deleteInstantFile(instant); + } + + @Override + public void deleteEmptyInstantIfExists(HoodieInstant instant) { + ValidationUtils.checkArgument(isEmpty(instant)); + deleteInstantFileIfExists(instant); + } + + @Override + public void deleteCompactionRequested(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isRequested()); + ValidationUtils.checkArgument(Objects.equals(instant.getAction(), HoodieTimeline.COMPACTION_ACTION)); + deleteInstantFile(instant); + } + + @Override + public void deleteInstantFileIfExists(HoodieInstant instant) { + LOG.info("Deleting instant " + instant); + StoragePath commitFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(instant)); + try { + if (metaClient.getStorage().exists(commitFilePath)) { + boolean result = metaClient.getStorage().deleteFile(commitFilePath); + if (result) { + LOG.info("Removed instant " + instant); + } else { + throw new HoodieIOException("Could not delete instant " + instant + " with path " + commitFilePath); + } + } else { + LOG.warn("The commit " + commitFilePath + " to remove does not exist"); + } + } catch (IOException e) { + throw new HoodieIOException("Could not remove commit " + commitFilePath, e); + } + } + + private void deleteInstantFile(HoodieInstant instant) { + LOG.info("Deleting instant " + instant); + StoragePath inFlightCommitFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(instant)); + try { + boolean result = metaClient.getStorage().deleteFile(inFlightCommitFilePath); + if (result) { + LOG.info("Removed instant " + instant); + } else { + throw new HoodieIOException("Could not delete instant " + instant + " with path " + inFlightCommitFilePath); + } + } catch (IOException e) { + throw new HoodieIOException("Could not remove inflight commit " + inFlightCommitFilePath, e); + } + } + + @Override + public Option getInstantDetails(HoodieInstant instant) { + StoragePath detailPath = getInstantFileNamePath(instantFileNameGenerator.getFileName(instant)); + return readDataFromPath(detailPath); + } + + @Override + public Option> getLastCommitMetadataWithValidSchema() { + return Option.fromJavaOptional( + getCommitMetadataStream() + .filter(instantCommitMetadataPair -> + WriteOperationType.canUpdateSchema(instantCommitMetadataPair.getRight().getOperationType()) + && !StringUtils.isNullOrEmpty(instantCommitMetadataPair.getValue().getMetadata(HoodieCommitMetadata.SCHEMA_KEY))) + .findFirst() + ); + } + + @Override + public Option> getLastCommitMetadataWithValidData() { + return Option.fromJavaOptional( + getCommitMetadataStream() + .filter(instantCommitMetadataPair -> + !instantCommitMetadataPair.getValue().getFileIdAndRelativePaths().isEmpty()) + .findFirst() + ); + } + + private Stream> getCommitMetadataStream() { + // NOTE: Streams are lazy + return getCommitsTimeline().filterCompletedInstants() + .getInstantsAsStream() + .sorted(Comparator.comparing(HoodieInstant::requestedTime).reversed()) + .map(instant -> { + try { + HoodieCommitMetadata commitMetadata = + metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantDetails(instant).get(), HoodieCommitMetadata.class); + return Pair.of(instant, commitMetadata); + } catch (IOException e) { + throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); + } + }); + } + + @Override + public Option readCleanerInfoAsBytes(HoodieInstant instant) { + // Cleaner metadata are always stored only in timeline .hoodie + return readDataFromPath(getInstantFileNamePath(instantFileNameGenerator.getFileName(instant))); + } + + @Override + public Option readRollbackInfoAsBytes(HoodieInstant instant) { + // Rollback metadata are always stored only in timeline .hoodie + return readDataFromPath(getInstantFileNamePath(instantFileNameGenerator.getFileName(instant))); + } + + @Override + public Option readRestoreInfoAsBytes(HoodieInstant instant) { + // Rollback metadata are always stored only in timeline .hoodie + return readDataFromPath(new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(instant))); + } + + //----------------------------------------------------------------- + // BEGIN - COMPACTION RELATED META-DATA MANAGEMENT. + //----------------------------------------------------------------- + @Override + public Option readCompactionPlanAsBytes(HoodieInstant instant) { + return readDataFromPath(new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(instant))); + } + + @Override + public Option readIndexPlanAsBytes(HoodieInstant instant) { + return readDataFromPath(new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(instant))); + } + + @Override + public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, inflightInstant.getAction(), inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, LOG_COMPACTION_ACTION, inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflightInstant, Option.empty()); + return inflightInstant; + } + + @Override + public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, LOG_COMPACTION_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflightInstant, Option.empty()); + return inflightInstant; + } + + @Override + public HoodieInstant transitionCompactionInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, + Option data) { + // Lock is not honored in 0.x mode. + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, COMMIT_ACTION, inflightInstant.requestedTime()); + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionLogCompactionInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + // Lock is not honored in 0.x mode. + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, DELTA_COMMIT_ACTION, inflightInstant.requestedTime()); + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + //----------------------------------------------------------------- + // END - COMPACTION RELATED META-DATA MANAGEMENT + //----------------------------------------------------------------- + + @Override + public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, + Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, CLEAN_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, CLEAN_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflight, data); + return inflight; + } + + @Override + public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, ROLLBACK_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, ROLLBACK_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflight, Option.empty()); + return inflight; + } + + @Override + public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.RESTORE_ACTION), "Transition to inflight requested for a restore instant with diff action " + + requestedInstant.toString()); + ValidationUtils.checkArgument(requestedInstant.isRequested(), "Transition to inflight requested for an instant not in requested state " + requestedInstant.toString()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, RESTORE_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflight, Option.empty()); + return inflight; + } + + @Override + public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.requestedTime()); + // Then write to timeline + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + @Override + public HoodieInstant transitionClusterRequestedToInflight(HoodieInstant requestedInstant, Option data) { + // In 0.x, no separate clustering action, reuse replace action. + return transitionReplaceRequestedToInflight(requestedInstant, data); + } + + @Override + public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, Option data) { + // In 0.x, no separate clustering action, reuse replace action. + return transitionReplaceInflightToComplete(shouldLock, inflightInstant, data); + } + + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { + transitionState(fromInstant, toInstant, data, false); + } + + protected void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, + boolean allowRedundantTransitions) { + ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); + try { + HoodieStorage storage = metaClient.getStorage(); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Re-create the .inflight file by opening a new file and write the commit metadata in + createFileInMetaPath(instantFileNameGenerator.getFileName(fromInstant), data, allowRedundantTransitions); + StoragePath fromInstantPath = getInstantFileNamePath(instantFileNameGenerator.getFileName(fromInstant)); + StoragePath toInstantPath = getInstantFileNamePath(instantFileNameGenerator.getFileName(toInstant)); + boolean success = storage.rename(fromInstantPath, toInstantPath); + if (!success) { + throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath); + } + } else { + // Ensures old state exists in timeline + ValidationUtils.checkArgument(storage.exists(getInstantFileNamePath(instantFileNameGenerator.getFileName(fromInstant))), + "File " + getInstantFileNamePath(instantFileNameGenerator.getFileName(fromInstant)) + " does not exist!"); + // Use Write Once to create Target File + if (allowRedundantTransitions) { + FileIOUtils.createFileInPath(storage, getInstantFileNamePath(instantFileNameGenerator.getFileName(toInstant)), data); + } else { + storage.createImmutableFileInPath(getInstantFileNamePath(instantFileNameGenerator.getFileName(toInstant)), data); + } + LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(instantFileNameGenerator.getFileName(toInstant))); + } + } catch (IOException e) { + throw new HoodieIOException("Could not complete " + fromInstant, e); + } + } + + protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { + ValidationUtils.checkArgument(completed.requestedTime().equals(inflight.requestedTime())); + StoragePath inFlightCommitFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(inflight)); + StoragePath commitFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(completed)); + try { + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + if (!metaClient.getStorage().exists(inFlightCommitFilePath)) { + boolean success = metaClient.getStorage().rename(commitFilePath, inFlightCommitFilePath); + if (!success) { + throw new HoodieIOException( + "Could not rename " + commitFilePath + " to " + inFlightCommitFilePath); + } + } + } else { + StoragePath requestedInstantFilePath = getInstantFileNamePath( + instantFileNameGenerator.getFileName(instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, + inflight.getAction(), inflight.requestedTime()))); + + // If inflight and requested files do not exist, create one + if (!metaClient.getStorage().exists(requestedInstantFilePath)) { + metaClient.getStorage().create(requestedInstantFilePath, false).close(); + } + + if (!metaClient.getStorage().exists(inFlightCommitFilePath)) { + metaClient.getStorage().create(inFlightCommitFilePath, false).close(); + } + + boolean success = metaClient.getStorage().deleteFile(commitFilePath); + ValidationUtils.checkArgument(success, "State Reverting failed"); + } + } catch (IOException e) { + throw new HoodieIOException("Could not complete revert " + completed, e); + } + } + + private StoragePath getInstantFileNamePath(String fileName) { + return new StoragePath(fileName.contains(SCHEMA_COMMIT_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath().toString(), fileName); + } + + @Override + public void transitionRequestedToInflight(String commitType, String inFlightInstant) { + HoodieInstant requested = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant); + transitionRequestedToInflight(requested, Option.empty(), false); + } + + @Override + public void transitionRequestedToInflight(HoodieInstant requested, Option content) { + transitionRequestedToInflight(requested, content, false); + } + + @Override + public void transitionRequestedToInflight(HoodieInstant requested, Option content, + boolean allowRedundantTransitions) { + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, requested.getAction(), requested.requestedTime()); + ValidationUtils.checkArgument(requested.isRequested(), "Instant " + requested + " in wrong state"); + transitionState(requested, inflight, content, allowRedundantTransitions); + } + + @Override + public void saveToCompactionRequested(HoodieInstant instant, Option content) { + saveToCompactionRequested(instant, content, false); + } + + @Override + public void saveToCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, overwrite); + } + + @Override + public void saveToLogCompactionRequested(HoodieInstant instant, Option content) { + saveToLogCompactionRequested(instant, content, false); + } + + @Override + public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, overwrite); + } + + @Override + public void saveToPendingReplaceCommit(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToPendingClusterCommit(HoodieInstant instant, Option content) { + // In 0.x, no separate clustering action, reuse replace action. + saveToPendingReplaceCommit(instant, content); + } + + @Override + public void saveToCleanRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToRollbackRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToRestoreRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested(), + String.format("Instant %s not in requested state", requestedInstant.requestedTime())); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, INDEXING_ACTION, requestedInstant.requestedTime()); + transitionState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + @Override + public HoodieInstant transitionIndexInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.requestedTime())); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, INDEXING_ACTION, inflightInstant.requestedTime()); + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.requestedTime())); + HoodieInstant requestedInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, INDEXING_ACTION, inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + transitionState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public void saveToPendingIndexAction(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", instant.getAction(), INDEXING_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + public void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { + StoragePath fullPath = getInstantFileNamePath(filename); + if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { + FileIOUtils.createFileInPath(metaClient.getStorage(), fullPath, content); + } else { + metaClient.getStorage().createImmutableFileInPath(fullPath, content); + } + } + + protected Option readDataFromPath(StoragePath detailPath) { + try (InputStream is = metaClient.getStorage().open(detailPath)) { + return Option.of(FileIOUtils.readAsByteArray(is)); + } catch (IOException e) { + throw new HoodieIOException("Could not read commit details from " + detailPath, e); + } + } + + @Override + public HoodieActiveTimeline reload() { + return new ActiveTimelineV1(metaClient); + } + + @Override + public void copyInstant(HoodieInstant instant, StoragePath dstDir) { + StoragePath srcPath = new StoragePath(metaClient.getMetaPath(), instantFileNameGenerator.getFileName(instant)); + StoragePath dstPath = new StoragePath(dstDir, instantFileNameGenerator.getFileName(instant)); + try { + HoodieStorage storage = metaClient.getStorage(); + storage.createDirectory(dstDir); + FileIOUtils.copy(storage, srcPath, storage, dstPath, false, true); + } catch (IOException e) { + throw new HoodieIOException("Could not copy instant from " + srcPath + " to " + dstPath, e); + } + } + + @Override + public Set getValidExtensions() { + return Collections.emptySet(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineLoaderV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineLoaderV1.java new file mode 100644 index 000000000000..03271efbc1a1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineLoaderV1.java @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.apache.hudi.storage.StoragePathInfo; + +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.jetbrains.annotations.Nullable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.Comparator; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.Spliterator; +import java.util.Spliterators; +import java.util.function.BiConsumer; +import java.util.function.Function; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.StreamSupport; + +public class ArchivedTimelineLoaderV1 implements ArchivedTimelineLoader { + private static final String MERGE_ARCHIVE_PLAN_NAME = "mergeArchivePlan"; + private static final Pattern ARCHIVE_FILE_PATTERN = + Pattern.compile("^\\.commits_\\.archive\\.([0-9]+).*"); + private static final String STATE_TRANSITION_TIME = "stateTransitionTime"; + private static final String ACTION_TYPE_KEY = "actionType"; + private static final Logger LOG = LoggerFactory.getLogger(ArchivedTimelineLoaderV1.class); + + @Override + public void loadInstants(HoodieTableMetaClient metaClient, + @Nullable HoodieArchivedTimeline.TimeRangeFilter filter, + HoodieArchivedTimeline.LoadMode loadMode, + Function commitsFilter, + BiConsumer recordConsumer) { + Set instantsInRange = new HashSet<>(); + try { + // List all files + List entryList = metaClient.getStorage().globEntries( + new StoragePath(metaClient.getArchivePath() + "/.commits_.archive*")); + + // Sort files by version suffix in reverse (implies reverse chronological order) + entryList.sort(new ArchiveFileVersionComparator()); + + for (StoragePathInfo fs : entryList) { + // Read the archived file + try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getStorage(), + new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) { + int instantsInPreviousFile = instantsInRange.size(); + // Read the avro blocks + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieAvroDataBlock) { + HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block; + // TODO If we can store additional metadata in datablock, we can skip parsing records + // (such as startTime, endTime of records in the block) + try (ClosableIterator> itr = avroBlock.getRecordIterator(HoodieRecord.HoodieRecordType.AVRO)) { + StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true) + // Filter blocks in desired time window + .map(r -> (GenericRecord) r.getData()) + .filter(commitsFilter::apply) + .forEach(r -> { + String instantTime = r.get(HoodieTableMetaClient.COMMIT_TIME_KEY).toString(); + instantsInRange.add(instantTime); + recordConsumer.accept(instantTime, r); + }); + } + } + } + + if (filter != null) { + int instantsInCurrentFile = instantsInRange.size() - instantsInPreviousFile; + if (instantsInPreviousFile > 0 && instantsInCurrentFile == 0) { + // Note that this is an optimization to skip reading unnecessary archived files + // This signals we crossed lower bound of desired time window. + break; + } + } + } catch (Exception originalException) { + // merge small archive files may left uncompleted archive file which will cause exception. + // need to ignore this kind of exception here. + try { + StoragePath planPath = new StoragePath(metaClient.getArchivePath(), MERGE_ARCHIVE_PLAN_NAME); + HoodieStorage storage = metaClient.getStorage(); + if (storage.exists(planPath)) { + HoodieMergeArchiveFilePlan plan = TimelineMetadataUtils.deserializeAvroMetadata(FileIOUtils.readDataFromPath(storage, planPath).get(), HoodieMergeArchiveFilePlan.class); + String mergedArchiveFileName = plan.getMergedArchiveFileName(); + if (!StringUtils.isNullOrEmpty(mergedArchiveFileName) && fs.getPath().getName().equalsIgnoreCase(mergedArchiveFileName)) { + LOG.warn("Catch exception because of reading uncompleted merging archive file " + mergedArchiveFileName + ". Ignore it here."); + continue; + } + } + throw originalException; + } catch (Exception e) { + // If anything wrong during parsing merge archive plan, we need to throw the original exception. + // For example corrupted archive file and corrupted plan are both existed. + throw originalException; + } + } + } + } catch (IOException e) { + throw new HoodieIOException( + "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); + } + } + + /** + * Sort files by reverse order of version suffix in file name. + */ + public static class ArchiveFileVersionComparator implements Comparator, Serializable { + @Override + public int compare(StoragePathInfo f1, StoragePathInfo f2) { + return Integer.compare(getArchivedFileSuffix(f2), getArchivedFileSuffix(f1)); + } + + private int getArchivedFileSuffix(StoragePathInfo f) { + try { + Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(f.getPath().getName()); + if (fileMatcher.matches()) { + return Integer.parseInt(fileMatcher.group(1)); + } + } catch (NumberFormatException e) { + // log and ignore any format warnings + LOG.warn("error getting suffix for archived file: " + f.getPath()); + } + + // return default value in case of any errors + return 0; + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineV1.java new file mode 100644 index 000000000000..85fa99a70528 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/ArchivedTimelineV1.java @@ -0,0 +1,251 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.storage.StoragePath; + +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.BiConsumer; +import java.util.function.Function; + +public class ArchivedTimelineV1 extends BaseTimelineV1 implements HoodieArchivedTimeline { + private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits"; + private static final String ACTION_TYPE_KEY = "actionType"; + private static final String ACTION_STATE = "actionState"; + private static final String STATE_TRANSITION_TIME = "stateTransitionTime"; + private HoodieTableMetaClient metaClient; + private final Map readCommits = new HashMap<>(); + private final ArchivedTimelineLoader timelineLoader = new ArchivedTimelineLoaderV1(); + + private static final Logger LOG = LoggerFactory.getLogger(org.apache.hudi.common.table.timeline.HoodieArchivedTimeline.class); + + /** + * Loads all the archived instants. + * Note that there is no lazy loading, so this may not work if the archived timeline range is really long. + * TBD: Should we enforce maximum time range? + */ + public ArchivedTimelineV1(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + setInstants(this.loadInstants(false)); + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + } + + /** + * Loads completed instants from startTs(inclusive). + * Note that there is no lazy loading, so this may not work if really early startTs is specified. + */ + public ArchivedTimelineV1(HoodieTableMetaClient metaClient, String startTs) { + this.metaClient = metaClient; + setInstants(loadInstants(new StartTsFilter(startTs), true, + record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()))); + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + public ArchivedTimelineV1() { + } + + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } + + @Override + public Option getInstantDetails(HoodieInstant instant) { + return Option.ofNullable(readCommits.get(instant.requestedTime())); + } + + public static StoragePath getArchiveLogPath(String archiveFolder) { + return new StoragePath(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX); + } + + @Override + public void loadInstantDetailsInMemory(String startTs, String endTs) { + loadInstants(startTs, endTs); + } + + @Override + public void loadCompletedInstantDetailsInMemory() { + loadInstants(null, true, + record -> { + // Very old archived instants don't have action state set. + Object action = record.get(ACTION_STATE); + return action == null || org.apache.hudi.common.table.timeline.HoodieInstant.State.COMPLETED.toString().equals(action.toString()); + }); + } + + @Override + public void loadCompactionDetailsInMemory(String compactionInstantTime) { + loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime); + } + + @Override + public void loadCompactionDetailsInMemory(String startTs, String endTs) { + // load compactionPlan + loadInstants(new HoodieArchivedTimeline.TimeRangeFilter(startTs, endTs), true, + record -> { + // Older files don't have action state set. + Object action = record.get(ACTION_STATE); + return record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION) + && (action == null || org.apache.hudi.common.table.timeline.HoodieInstant.State.INFLIGHT.toString().equals(action.toString())); + }); + } + + @Override + public void clearInstantDetailsFromMemory(String instantTime) { + this.readCommits.remove(instantTime); + } + + @Override + public void clearInstantDetailsFromMemory(String startTs, String endTs) { + this.findInstantsInRange(startTs, endTs).getInstants().forEach(instant -> + this.readCommits.remove(instant.requestedTime())); + } + + private List loadInstants(boolean loadInstantDetails) { + return loadInstants(null, loadInstantDetails); + } + + private List loadInstants(String startTs, String endTs) { + return loadInstants(new HoodieArchivedTimeline.TimeRangeFilter(startTs, endTs), true); + } + + private List loadInstants(HoodieArchivedTimeline.TimeRangeFilter filter, boolean loadInstantDetails) { + return loadInstants(filter, loadInstantDetails, genericRecord -> true); + } + + private List loadInstants(HoodieArchivedTimeline.TimeRangeFilter filter, boolean loadInstantDetails, Function commitsFilter) { + InstantsLoader loader = new InstantsLoader(loadInstantDetails); + timelineLoader.loadInstants(metaClient, filter, LoadMode.PLAN, commitsFilter, loader); + List result = new ArrayList<>(loader.getInstantsInRangeCollected().values()); + Collections.sort(result); + return result; + } + + /** + * Callback to read instant details. + */ + private class InstantsLoader implements BiConsumer { + private final Map instantsInRange = new ConcurrentHashMap<>(); + private final boolean loadInstantDetails; + + private InstantsLoader(boolean loadInstantDetails) { + this.loadInstantDetails = loadInstantDetails; + } + + @Override + public void accept(String instantTime, GenericRecord record) { + HoodieInstant instant = readCommit(instantTime, record, loadInstantDetails); + instantsInRange.putIfAbsent(instant.requestedTime(), instant); + } + + public Map getInstantsInRangeCollected() { + return instantsInRange; + } + } + + private HoodieInstant readCommit(String instantTime, GenericRecord record, boolean loadDetails) { + final String action = record.get(ACTION_TYPE_KEY).toString(); + final String stateTransitionTime = (String) record.get(STATE_TRANSITION_TIME); + if (loadDetails) { + getMetadataKey(action).map(key -> { + Object actionData = record.get(key); + if (actionData != null) { + if (action.equals(HoodieTimeline.COMPACTION_ACTION)) { + this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData)); + } else { + this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8)); + } + } + return null; + }); + } + return instantGenerator.createNewInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, + instantTime, stateTransitionTime); + } + + @Nonnull + private Option getMetadataKey(String action) { + switch (action) { + case HoodieTimeline.CLEAN_ACTION: + return Option.of("hoodieCleanMetadata"); + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: + return Option.of("hoodieCommitMetadata"); + case HoodieTimeline.ROLLBACK_ACTION: + return Option.of("hoodieRollbackMetadata"); + case HoodieTimeline.SAVEPOINT_ACTION: + return Option.of("hoodieSavePointMetadata"); + case HoodieTimeline.COMPACTION_ACTION: + case HoodieTimeline.LOG_COMPACTION_ACTION: + return Option.of("hoodieCompactionPlan"); + case HoodieTimeline.REPLACE_COMMIT_ACTION: + return Option.of("hoodieReplaceCommitMetadata"); + case HoodieTimeline.INDEXING_ACTION: + return Option.of("hoodieIndexCommitMetadata"); + default: + LOG.error(String.format("Unknown action in metadata (%s)", action)); + return Option.empty(); + } + } + + @Override + public HoodieArchivedTimeline reload() { + return new ArchivedTimelineV1(metaClient); + } + + @Override + public HoodieArchivedTimeline reload(String startTs) { + return new ArchivedTimelineV1(metaClient, startTs); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/BaseTimelineV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/BaseTimelineV1.java new file mode 100644 index 000000000000..f8b4c5b190af --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/BaseTimelineV1.java @@ -0,0 +1,153 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.timeline.BaseHoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; + +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class BaseTimelineV1 extends BaseHoodieTimeline { + + public BaseTimelineV1(Stream instants, Function> details) { + this(instants, details, TimelineLayout.fromVersion(TimelineLayoutVersion.LAYOUT_VERSION_1)); + } + + private BaseTimelineV1(Stream instants, Function> details, TimelineLayout layout) { + super(instants, details, layout.getTimelineFactory(), layout.getInstantComparator(), layout.getInstantGenerator()); + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + @Deprecated + public BaseTimelineV1() { + super(TimelineLayout.fromVersion(TimelineLayoutVersion.LAYOUT_VERSION_1)); + } + + @Override + public HoodieTimeline getWriteTimeline() { + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); + } + + @Override + public HoodieTimeline filterPendingClusteringTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && !s.isCompleted()) + .filter(i -> ClusteringUtils.isClusteringInstant(this, i, instantGenerator)), details); + } + + @Override + public HoodieTimeline filterPendingReplaceOrClusteringTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> (s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) + && !s.isCompleted()), details); + } + + @Override + public HoodieTimeline filterPendingReplaceClusteringAndCompactionTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> !s.isCompleted() && (s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) + || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); + } + + @Override + public HoodieTimeline getCommitsTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION)); + } + + @Override + public HoodieTimeline getCommitAndReplaceTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION)); + } + + @Override + public TimelineLayoutVersion getTimelineLayoutVersion() { + return TimelineLayoutVersion.LAYOUT_VERSION_1; + } + + @Override + public Option getLastClusteringInstant() { + return Option.fromJavaOptional(getCommitsTimeline().filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION)) + .getReverseOrderedInstants() + .filter(i -> ClusteringUtils.isClusteringInstant(this, i, instantGenerator)) + .findFirst()); + } + + @Override + public Option getFirstPendingClusterInstant() { + return getLastOrFirstPendingClusterInstant(false); + } + + @Override + public Option getLastPendingClusterInstant() { + return getLastOrFirstPendingClusterInstant(true); + } + + private Option getLastOrFirstPendingClusterInstant(boolean isLast) { + HoodieTimeline replaceTimeline = filterPendingReplaceTimeline(); + Stream replaceStream; + if (isLast) { + replaceStream = replaceTimeline.getReverseOrderedInstants(); + } else { + replaceStream = replaceTimeline.getInstantsAsStream(); + } + return Option.fromJavaOptional(replaceStream + .filter(i -> ClusteringUtils.isClusteringInstant(this, i, instantGenerator)).findFirst()); + } + + @Override + public boolean isPendingClusteringInstant(String instantTime) { + return getOrCreatePendingClusteringInstantSet().contains(instantTime); + } + + private Set getOrCreatePendingClusteringInstantSet() { + if (this.pendingClusteringInstants == null) { + synchronized (this) { + if (this.pendingClusteringInstants == null) { + List pendingReplaceInstants = getCommitsTimeline().filterPendingReplaceTimeline().getInstants(); + // Validate that there are no instants with same timestamp + pendingReplaceInstants.stream().collect(Collectors.groupingBy(HoodieInstant::requestedTime)).forEach((timestamp, instants) -> { + if (instants.size() > 1) { + throw new IllegalStateException("Multiple instants with same timestamp: " + timestamp + " instants: " + instants); + } + }); + // Filter replace commits down to those that are due to clustering + this.pendingClusteringInstants = pendingReplaceInstants.stream() + .filter(instant -> ClusteringUtils.isClusteringInstant(this, instant, instantGenerator)) + .map(HoodieInstant::requestedTime).collect(Collectors.toSet()); + } + } + } + return this.pendingClusteringInstants; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CommitMetadataSerDeV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CommitMetadataSerDeV1.java new file mode 100644 index 000000000000..48567a64ffd0 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CommitMetadataSerDeV1.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.JsonUtils; +import org.apache.hudi.common.util.Option; + +import java.io.IOException; + +import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes; + +public class CommitMetadataSerDeV1 implements CommitMetadataSerDe { + + @Override + public T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException { + try { + if (bytes.length == 0) { + return clazz.newInstance(); + } + return fromJsonString(fromUTF8Bytes(bytes), clazz); + } catch (Exception e) { + throw new IOException("unable to read commit metadata for instant " + instant + " bytes length: " + bytes.length, e); + } + } + + public static T fromJsonString(String jsonStr, Class clazz) throws Exception { + if (jsonStr == null || jsonStr.isEmpty()) { + // For empty commit file + return clazz.newInstance(); + } + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); + } + + @Override + public Option serialize(HoodieCommitMetadata commitMetadata) throws IOException { + return Option.ofNullable(commitMetadata.toJsonString().getBytes()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CompletionTimeQueryViewV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CompletionTimeQueryViewV1.java new file mode 100644 index 000000000000..370a263469bc --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/CompletionTimeQueryViewV1.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; + +import java.io.Serializable; +import java.time.Instant; +import java.util.Date; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; + +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; + +public class CompletionTimeQueryViewV1 implements CompletionTimeQueryView, Serializable { + private static final long serialVersionUID = 1L; + + private static final long MILLI_SECONDS_IN_THREE_DAYS = 3 * 24 * 3600 * 1000; + + private static final long MILLI_SECONDS_IN_ONE_DAY = 24 * 3600 * 1000; + + private final HoodieTableMetaClient metaClient; + + /** + * Mapping from instant start time -> completion time. + * Should be thread-safe data structure. + */ + private final ConcurrentMap beginToCompletionInstantTimeMap; + + /** + * The cursor instant time to eagerly load from, by default load last N days of completed instants. + * It can grow dynamically with lazy loading. e.g. assuming an initial cursor instant as t10, + * a completion query for t5 would trigger lazy loading with this cursor instant updated to t5. + * This sliding window model amortizes redundant loading from different queries. + */ + private volatile String cursorInstant; + + /** + * The first write instant on the active timeline, used for query optimization. + */ + private final String firstNonSavepointCommit; + + /** + * The constructor. + * + * @param metaClient The table meta client. + */ + public CompletionTimeQueryViewV1(HoodieTableMetaClient metaClient) { + this(metaClient, HoodieInstantTimeGenerator.formatDate(new Date(Instant.now().minusMillis(MILLI_SECONDS_IN_THREE_DAYS).toEpochMilli()))); + } + + /** + * The constructor. + * + * @param metaClient The table meta client. + * @param eagerLoadInstant The earliest instant time to eagerly load from, by default load last N days of completed instants. + */ + public CompletionTimeQueryViewV1(HoodieTableMetaClient metaClient, String eagerLoadInstant) { + this.metaClient = metaClient; + this.beginToCompletionInstantTimeMap = new ConcurrentHashMap<>(); + this.cursorInstant = InstantComparison.minInstant(eagerLoadInstant, metaClient.getActiveTimeline().firstInstant().map(HoodieInstant::requestedTime).orElse("")); + // Note: use getWriteTimeline() to keep sync with the fs view visibleCommitsAndCompactionTimeline, see AbstractTableFileSystemView.refreshTimeline. + this.firstNonSavepointCommit = metaClient.getActiveTimeline().getWriteTimeline().getFirstNonSavepointCommit().map(HoodieInstant::requestedTime).orElse(""); + load(); + } + + @Override + public boolean isCompleted(String beginInstantTime) { + // archival does not proceed beyond the first savepoint, so any instant before that is completed. + return this.beginToCompletionInstantTimeMap.containsKey(beginInstantTime) || isArchived(beginInstantTime); + } + + @Override + public boolean isArchived(String instantTime) { + return InstantComparison.compareTimestamps(instantTime, LESSER_THAN, this.firstNonSavepointCommit); + } + + @Override + public boolean isCompletedBefore(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); + if (completionTimeOpt.isPresent()) { + return InstantComparison.compareTimestamps(completionTimeOpt.get(), LESSER_THAN, baseInstant); + } + return false; + } + + @Override + public boolean isSlicedAfterOrOn(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); + if (completionTimeOpt.isPresent()) { + return InstantComparison.compareTimestamps(completionTimeOpt.get(), GREATER_THAN_OR_EQUALS, baseInstant); + } + return true; + } + + @Override + public Option getCompletionTime(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(instantTime); + if (completionTimeOpt.isPresent()) { + String completionTime = completionTimeOpt.get(); + if (completionTime.length() != baseInstant.length()) { + // ============================================================== + // LEGACY CODE + // ============================================================== + // Fixes the completion time to reflect the completion sequence correctly + // if the file slice base instant time is not in datetime format. + // For example, many test cases just use integer string as the instant time. + // CAUTION: this fix only works for OCC(Optimistic Concurrency Control). + // for NB-CC(Non-blocking Concurrency Control), the file slicing may be incorrect. + return Option.of(instantTime); + } + } + return completionTimeOpt; + } + + @Override + public Option getCompletionTime(String beginTime) { + String completionTime = this.beginToCompletionInstantTimeMap.get(beginTime); + if (completionTime != null) { + return Option.of(completionTime); + } + + // ***This is the key change between V1 and V2 completion time query-view*** + if (isArchived(beginTime)) { + // Completion time and begin time are same for archived instants. + return Option.of(beginTime); + } + // the instant is still pending + return Option.empty(); + } + + @Override + public List getInstantTimes( + HoodieTimeline timeline, + Option rangeStart, + Option rangeEnd, + InstantRange.RangeType rangeType) { + // assumes any instant/transaction lasts at most 1 day to optimize the query efficiency. + return getInstantTimes(timeline, rangeStart, rangeEnd, rangeType, s -> HoodieInstantTimeGenerator.instantTimeMinusMillis(s, MILLI_SECONDS_IN_ONE_DAY)); + } + + @Override + @VisibleForTesting + public List getInstantTimes( + String rangeStart, + String rangeEnd, + Function earliestInstantTimeFunc) { + return getInstantTimes(metaClient.getCommitsTimeline().filterCompletedInstants(), Option.ofNullable(rangeStart), Option.ofNullable(rangeEnd), + InstantRange.RangeType.CLOSED_CLOSED, earliestInstantTimeFunc); + } + + /** + * Queries the instant start time with given completion time range. + * + * @param timeline The timeline. + * @param rangeStart The query range start completion time. + * @param rangeEnd The query range end completion time. + * @param rangeType The range type. + * @param earliestInstantTimeFunc The function to generate the earliest start time boundary + * with the minimum completion time. + * + * @return The sorted instant time list. + */ + private List getInstantTimes( + HoodieTimeline timeline, + Option rangeStart, + Option rangeEnd, + InstantRange.RangeType rangeType, + Function earliestInstantTimeFunc) { + throw new RuntimeException("Incremental query view for timeline version 1 not yet implemented"); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + /** + * This is method to read instant completion time. + * This would also update 'startToCompletionInstantTimeMap' map with start time/completion time pairs. + * Only instants starts from 'startInstant' (inclusive) are considered. + */ + private void load() { + // load active instants first. + this.metaClient.getActiveTimeline() + .filterCompletedInstants().getInstantsAsStream() + .forEach(instant -> setCompletionTime(instant.requestedTime(), instant.getCompletionTime())); + // Do not load archive timeline. + } + + private void setCompletionTime(String beginInstantTime, String completionTime) { + if (completionTime == null) { + // the meta-server instant does not have completion time + completionTime = beginInstantTime; + } + this.beginToCompletionInstantTimeMap.putIfAbsent(beginInstantTime, completionTime); + } + + @Override + public String getCursorInstant() { + return cursorInstant; + } + + @Override + public boolean isEmptyTable() { + return this.beginToCompletionInstantTimeMap.isEmpty(); + } + + @Override + public void close() { + this.beginToCompletionInstantTimeMap.clear(); + } +} \ No newline at end of file diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantComparatorV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantComparatorV1.java new file mode 100644 index 000000000000..8b21e672b797 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantComparatorV1.java @@ -0,0 +1,77 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantComparator; +import org.apache.hudi.common.table.timeline.versioning.common.InstantComparators; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; + +public class InstantComparatorV1 implements Serializable, InstantComparator { + + /** + * A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants + * for state transitions, this needs to be taken into account + */ + private static final Map COMPARABLE_ACTIONS = createComparableActionsMap(); + + public static final Comparator ACTION_COMPARATOR = + new InstantComparators.ActionComparator(COMPARABLE_ACTIONS); + + public static final Comparator REQUESTED_TIME_BASED_COMPARATOR = + new InstantComparators.RequestedTimeBasedComparator(COMPARABLE_ACTIONS); + + public static final Comparator COMPLETION_TIME_BASED_COMPARATOR = + new InstantComparators.CompletionTimeBasedComparator(COMPARABLE_ACTIONS); + + public static String getComparableAction(String action) { + return COMPARABLE_ACTIONS.getOrDefault(action, action); + } + + private static final Map createComparableActionsMap() { + Map comparableMap = new HashMap<>(); + comparableMap.put(COMPACTION_ACTION, COMMIT_ACTION); + comparableMap.put(LOG_COMPACTION_ACTION, DELTA_COMMIT_ACTION); + return comparableMap; + } + + @Override + public Comparator actionOnlyComparator() { + return ACTION_COMPARATOR; + } + + @Override + public Comparator requestedTimeOrderedComparator() { + return REQUESTED_TIME_BASED_COMPARATOR; + } + + @Override + public Comparator completionTimeOrderedComparator() { + return COMPLETION_TIME_BASED_COMPARATOR; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameGeneratorV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameGeneratorV1.java new file mode 100644 index 000000000000..cc5897bc2ab1 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameGeneratorV1.java @@ -0,0 +1,273 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; + +/** + * + */ +public class InstantFileNameGeneratorV1 implements InstantFileNameGenerator { + + @Override + public String makeCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.COMMIT_EXTENSION); + } + + @Override + public String makeInflightCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION); + } + + @Override + public String makeCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION); + } + + @Override + public String makeRequestedCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLEAN_EXTENSION); + } + + @Override + public String makeInflightCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION); + } + + @Override + public String makeRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.ROLLBACK_EXTENSION); + } + + @Override + public String makeRequestedRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_ROLLBACK_EXTENSION); + } + + @Override + public String makeRequestedRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_RESTORE_EXTENSION); + } + + @Override + public String makeInflightRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); + } + + @Override + public String makeInflightSavePointFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); + } + + @Override + public String makeSavePointFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.SAVEPOINT_EXTENSION); + } + + @Override + public String makeInflightDeltaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedDeltaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION); + } + + @Override + public String makeInflightCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); + } + + @Override + public String makeRequestedCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); + } + + @Override + public String makeInflightLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_LOG_COMPACTION_EXTENSION); + } + + @Override + public String makeRequestedLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_LOG_COMPACTION_EXTENSION); + } + + @Override + public String makeRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION); + } + + @Override + public String makeInflightRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION); + } + + @Override + public String makeReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeInflightReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeDeltaFileName(String instantTime) { + return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION; + } + + @Override + public String getCommitFromCommitFile(String commitFileName) { + return commitFileName.split("\\.")[0]; + } + + @Override + public String makeFileNameAsComplete(String fileName) { + return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, ""); + } + + @Override + public String makeFileNameAsInflight(String fileName) { + return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); + } + + @Override + public String makeIndexCommitFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeInflightIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION); + } + + @Override + public String makeInflightSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION); + } + + @Override + public String makeRequestSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION); + } + + @Override + public String makeRequestedClusteringFileName(String instant) { + // 1n 0.x, clustering and replace commit had the same filename + return makeRequestedReplaceFileName(instant); + } + + @Override + public String makeInflightClusteringFileName(String instant) { + // 1n 0.x, clustering and replace commit had the same filename + return makeInflightReplaceFileName(instant); + } + + @Override + public String getFileName(HoodieInstant instant) { + String action = instant.getAction(); + String timestamp = instant.requestedTime(); + if (HoodieTimeline.COMMIT_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightCommitFileName(timestamp) + : instant.isRequested() ? makeRequestedCommitFileName(timestamp) + : makeCommitFileName(timestamp); + } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightCleanerFileName(timestamp) + : instant.isRequested() ? makeRequestedCleanerFileName(timestamp) + : makeCleanerFileName(timestamp); + } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightRollbackFileName(timestamp) + : instant.isRequested() ? makeRequestedRollbackFileName(timestamp) + : makeRollbackFileName(timestamp); + } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightSavePointFileName(timestamp) + : makeSavePointFileName(timestamp); + } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightDeltaFileName(timestamp) + : instant.isRequested() ? makeRequestedDeltaFileName(timestamp) + : makeDeltaFileName(timestamp); + } else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightCompactionFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedCompactionFileName(timestamp); + } else { + return makeCommitFileName(timestamp); + } + } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightLogCompactionFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedLogCompactionFileName(timestamp); + } else { + return makeDeltaFileName(timestamp); + } + } else if (HoodieTimeline.RESTORE_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightRestoreFileName(timestamp) + : instant.isRequested() ? makeRequestedRestoreFileName(timestamp) + : makeRestoreFileName(timestamp); + } else if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightReplaceFileName(timestamp) + : instant.isRequested() ? makeRequestedReplaceFileName(timestamp) + : makeReplaceFileName(timestamp); + } else if (HoodieTimeline.INDEXING_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightIndexFileName(timestamp) + : instant.isRequested() ? makeRequestedIndexFileName(timestamp) + : makeIndexCommitFileName(timestamp); + } else if (HoodieTimeline.SCHEMA_COMMIT_ACTION.equals(action)) { + return instant.isInflight() ? makeInflightSchemaFileName(timestamp) + : instant.isRequested() ? makeRequestSchemaFileName(timestamp) + : makeSchemaFileName(timestamp); + } + throw new IllegalArgumentException("Cannot get file name for unknown action " + action); + } + + @Override + public String getFileName(String completionTime, HoodieInstant instant) { + // Completion Time does not matter for Hoodie 0.x + return getFileName(instant); + } +} + diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameParserV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameParserV1.java new file mode 100644 index 000000000000..f470efeb849b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantFileNameParserV1.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; + +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class InstantFileNameParserV1 implements InstantFileNameParser { + + private static final Pattern NAME_FORMAT = Pattern.compile("^(\\d+)(\\.\\w+)(\\.\\D+)?$"); + + public String extractTimestamp(String fileName) throws IllegalArgumentException { + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + return matcher.group(1); + } + throw new IllegalArgumentException("Failed to retrieve timestamp from name: " + + String.format(HoodieInstant.FILE_NAME_FORMAT_ERROR, fileName)); + } + + public String getTimelineFileExtension(String fileName) { + Objects.requireNonNull(fileName); + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + return fileName.substring(matcher.group(1).length()); + } + return HoodieInstant.EMPTY_FILE_EXTENSION; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantGeneratorV1.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantGeneratorV1.java new file mode 100644 index 000000000000..2ecf0f1d8683 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/InstantGeneratorV1.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.storage.StoragePathInfo; + +import java.util.Date; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class InstantGeneratorV1 implements InstantGenerator { + + private static final Pattern NAME_FORMAT = + Pattern.compile("^(\\d+)(\\.\\w+)(\\.\\D+)?$"); + + private static final String DELIMITER = "."; + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp) { + return new HoodieInstant(state, action, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime) { + return new HoodieInstant(state, action, timestamp, completionTime, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime, boolean isLegacy) { + return new HoodieInstant(state, action, timestamp, completionTime, isLegacy, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(StoragePathInfo pathInfo) { + // First read the instant timestamp. [==>20170101193025<==].commit + String fileName = pathInfo.getPath().getName(); + String timestamp = null; + String stateTransitionTime = null; + String action = null; + HoodieInstant.State state = HoodieInstant.State.NIL; + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + timestamp = matcher.group(1); + if (matcher.group(2).equals(org.apache.hudi.common.table.timeline.HoodieTimeline.INFLIGHT_EXTENSION)) { + // This is to support backwards compatibility on how in-flight commit files were written + // General rule is inflight extension is ..inflight, but for commit it is .inflight + action = HoodieTimeline.COMMIT_ACTION; + state = HoodieInstant.State.INFLIGHT; + } else { + action = matcher.group(2).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING); + if (matcher.groupCount() == 3 && matcher.group(3) != null) { + state = HoodieInstant.State.valueOf(matcher.group(3).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING).toUpperCase()); + } else { + // Like 20230104152218702.commit + state = HoodieInstant.State.COMPLETED; + } + } + stateTransitionTime = + HoodieInstantTimeGenerator.formatDate(new Date(pathInfo.getModificationTime())); + } else { + throw new IllegalArgumentException("Failed to construct HoodieInstant: " + String.format(HoodieInstant.FILE_NAME_FORMAT_ERROR, fileName)); + } + return new HoodieInstant(state, action, timestamp, stateTransitionTime, true, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getRequestedInstant(final HoodieInstant instant) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, instant.getAction(), instant.requestedTime(), InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCleanRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCleanInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getLogCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.LOG_COMPACTION_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getLogCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { + return instant.isRequested() ? instant : getRequestedInstant(instant); + } + + @Override + public HoodieInstant getRestoreRequestedInstant(HoodieInstant instant) { + return instant.isRequested() ? instant : getRequestedInstant(instant); + } + + @Override + public HoodieInstant getIndexRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.INDEXING_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getIndexInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.INDEXING_ACTION, timestamp, InstantComparatorV1.REQUESTED_TIME_BASED_COMPARATOR); + } + + /** + * Special Handling of 0.x mode in 1.x goes below + */ + + @Override + public HoodieInstant getClusteringCommitRequestedInstant(String timestamp) { + // 1n 0.x, clustering and replace commit had the same filename + return getReplaceCommitRequestedInstant(timestamp); + } + + @Override + public HoodieInstant getClusteringCommitInflightInstant(String timestamp) { + // 1n 0.x, clustering and replace commit had the same filename + return getReplaceCommitInflightInstant(timestamp); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/TimelineV1Factory.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/TimelineV1Factory.java new file mode 100644 index 000000000000..cc7795d1dacb --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v1/TimelineV1Factory.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v1; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.util.Option; + +import java.util.function.Function; +import java.util.stream.Stream; + +public class TimelineV1Factory extends TimelineFactory { + + private final TimelineLayout layout; + + public TimelineV1Factory(TimelineLayout layout) { + this.layout = layout; + } + + @Override + public HoodieTimeline createDefaultTimeline(Stream instants, Function> details) { + return new BaseTimelineV1(instants, details); + } + + @Override + public HoodieActiveTimeline createActiveTimeline() { + return new ActiveTimelineV1(); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient) { + return new ArchivedTimelineV1(metaClient); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient, String startTs) { + return new ArchivedTimelineV1(metaClient, startTs); + } + + @Override + public ArchivedTimelineLoader createArchivedTimelineLoader() { + return new ArchivedTimelineLoaderV1(); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient) { + return new ActiveTimelineV1(metaClient); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { + return new ActiveTimelineV1(metaClient, applyLayoutFilter); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient) { + return new CompletionTimeQueryViewV1(metaClient); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient, String eagerInstant) { + return new CompletionTimeQueryViewV1(metaClient, eagerInstant); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java new file mode 100644 index 000000000000..aeac5d605d64 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ActiveTimelineV2.java @@ -0,0 +1,776 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.TimeGenerator; +import org.apache.hudi.common.table.timeline.TimeGenerators; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.storage.HoodieStorage; +import org.apache.hudi.storage.StoragePath; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.io.Serializable; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.HashSet; +import java.util.Objects; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Stream; + +public class ActiveTimelineV2 extends BaseTimelineV2 implements HoodieActiveTimeline { + + public static final Set VALID_EXTENSIONS_IN_ACTIVE_TIMELINE = new HashSet<>(Arrays.asList( + COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, + DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, + SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, + CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, + INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, + REQUESTED_RESTORE_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + INFLIGHT_LOG_COMPACTION_EXTENSION, REQUESTED_LOG_COMPACTION_EXTENSION, + ROLLBACK_EXTENSION, REQUESTED_ROLLBACK_EXTENSION, INFLIGHT_ROLLBACK_EXTENSION, + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION, + REQUESTED_INDEX_COMMIT_EXTENSION, INFLIGHT_INDEX_COMMIT_EXTENSION, INDEX_COMMIT_EXTENSION, + REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION, INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION, SAVE_SCHEMA_ACTION_EXTENSION, + REQUESTED_CLUSTERING_COMMIT_EXTENSION, INFLIGHT_CLUSTERING_COMMIT_EXTENSION)); + + private static final Logger LOG = LoggerFactory.getLogger(ActiveTimelineV2.class); + protected HoodieTableMetaClient metaClient; + private final InstantFileNameGenerator instantFileNameGenerator = new InstantFileNameGeneratorV2(); + + private ActiveTimelineV2(HoodieTableMetaClient metaClient, Set includedExtensions, + boolean applyLayoutFilters) { + // Filter all the filter in the metapath and include only the extensions passed and + // convert them into HoodieInstant + try { + this.setInstants(metaClient.scanHoodieInstantsFromFileSystem(includedExtensions, applyLayoutFilters)); + } catch (IOException e) { + throw new HoodieIOException("Failed to scan metadata", e); + } + this.metaClient = metaClient; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + LOG.info("Loaded instants upto : " + lastInstant()); + } + + public ActiveTimelineV2(HoodieTableMetaClient metaClient) { + this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), true); + } + + public ActiveTimelineV2(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { + this(metaClient, Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE), applyLayoutFilter); + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + @Deprecated + public ActiveTimelineV2() { + } + + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + @Deprecated + private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } + + @Override + public Set getValidExtensionsInActiveTimeline() { + return Collections.unmodifiableSet(VALID_EXTENSIONS_IN_ACTIVE_TIMELINE); + } + + @Override + public void createCompleteInstant(HoodieInstant instant) { + LOG.info("Creating a new complete instant " + instant); + createCompleteFileInMetaPath(true, instant, Option.empty()); + } + + @Override + public void createNewInstant(HoodieInstant instant) { + LOG.info("Creating a new instant " + instant); + ValidationUtils.checkArgument(!instant.isCompleted()); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), Option.empty(), false); + } + + @Override + public void createRequestedCommitWithReplaceMetadata(String instantTime, String actionType) { + try { + HoodieInstant instant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, actionType, instantTime); + LOG.info("Creating a new instant " + instant); + // Create the request replace file + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), + TimelineMetadataUtils.serializeRequestedReplaceMetadata(new HoodieRequestedReplaceMetadata()), false); + } catch (IOException e) { + throw new HoodieIOException("Error create requested replace commit ", e); + } + } + + @Override + public void saveAsComplete(HoodieInstant instant, Option data) { + saveAsComplete(true, instant, data); + } + + @Override + public void saveAsComplete(boolean shouldLock, HoodieInstant instant, Option data) { + LOG.info("Marking instant complete " + instant); + ValidationUtils.checkArgument(instant.isInflight(), + "Could not mark an already completed instant as complete again " + instant); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, instant.getAction(), instant.requestedTime()); + transitionStateToComplete(shouldLock, instant, commitInstant, data); + LOG.info("Completed " + instant); + } + + @Override + public HoodieInstant revertToInflight(HoodieInstant instant) { + LOG.info("Reverting instant to inflight " + instant); + HoodieInstant inflight = TimelineUtils.getInflightInstant(instant, metaClient); + revertCompleteToInflight(instant, inflight); + LOG.info("Reverted " + instant + " to inflight " + inflight); + return inflight; + } + + @Override + public void deleteInflight(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isInflight()); + deleteInstantFile(instant); + } + + @Override + public void deletePending(HoodieInstant instant) { + ValidationUtils.checkArgument(!instant.isCompleted()); + deleteInstantFile(instant); + } + + @Override + public void deleteCompletedRollback(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isCompleted()); + ValidationUtils.checkArgument(Objects.equals(instant.getAction(), HoodieTimeline.ROLLBACK_ACTION)); + deleteInstantFile(instant); + } + + @Override + public void deleteEmptyInstantIfExists(HoodieInstant instant) { + ValidationUtils.checkArgument(isEmpty(instant)); + deleteInstantFileIfExists(instant); + } + + @Override + public void deleteCompactionRequested(HoodieInstant instant) { + ValidationUtils.checkArgument(instant.isRequested()); + ValidationUtils.checkArgument(Objects.equals(instant.getAction(), HoodieTimeline.COMPACTION_ACTION)); + deleteInstantFile(instant); + } + + @Override + public void deleteInstantFileIfExists(HoodieInstant instant) { + LOG.info("Deleting instant " + instant); + StoragePath commitFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(instant)); + try { + if (metaClient.getStorage().exists(commitFilePath)) { + boolean result = metaClient.getStorage().deleteFile(commitFilePath); + if (result) { + LOG.info("Removed instant " + instant); + } else { + throw new HoodieIOException("Could not delete instant " + instant + " with path " + commitFilePath); + } + } else { + LOG.warn("The commit " + commitFilePath + " to remove does not exist"); + } + } catch (IOException e) { + throw new HoodieIOException("Could not remove commit " + commitFilePath, e); + } + } + + protected void deleteInstantFile(HoodieInstant instant) { + LOG.info("Deleting instant " + instant); + StoragePath filePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(instant)); + try { + boolean result = metaClient.getStorage().deleteFile(filePath); + if (result) { + LOG.info("Removed instant " + instant); + } else { + throw new HoodieIOException("Could not delete instant " + instant + " with path " + filePath); + } + } catch (IOException e) { + throw new HoodieIOException("Could not remove inflight commit " + filePath, e); + } + } + + /** + * Many callers might not pass completionTime, here we have to search + * timeline to get completionTime, the impact should be minor since + * 1. It appeals only tests pass instant without completion time + * 2. we already holds all instants in memory, the cost should be minor. + * + *

TODO: [HUDI-6885] Depreciate HoodieActiveTimeline#getInstantFileName and fix related tests. + */ + protected String getInstantFileName(HoodieInstant instant) { + if (instant.isCompleted() && instant.getCompletionTime() == null) { + return instantFileNameGenerator.getFileName(getInstantsAsStream().filter(s -> s.equals(instant)) + .findFirst().orElseThrow(() -> new HoodieIOException("Cannot find the instant" + instant))); + } + return instantFileNameGenerator.getFileName(instant); + } + + @Override + public Option getInstantDetails(HoodieInstant instant) { + StoragePath detailPath = getInstantFileNamePath(getInstantFileName(instant)); + return readDataFromPath(detailPath); + } + + @Override + public Option> getLastCommitMetadataWithValidSchema() { + return Option.fromJavaOptional( + getCommitMetadataStream() + .filter(instantCommitMetadataPair -> + WriteOperationType.canUpdateSchema(instantCommitMetadataPair.getRight().getOperationType()) + && !StringUtils.isNullOrEmpty(instantCommitMetadataPair.getValue().getMetadata(HoodieCommitMetadata.SCHEMA_KEY))) + .findFirst() + ); + } + + @Override + public Option> getLastCommitMetadataWithValidData() { + return Option.fromJavaOptional( + getCommitMetadataStream() + .filter(instantCommitMetadataPair -> + !instantCommitMetadataPair.getValue().getFileIdAndRelativePaths().isEmpty()) + .findFirst() + ); + } + + private Stream> getCommitMetadataStream() { + // NOTE: Streams are lazy + return getCommitsTimeline().filterCompletedInstants() + .getInstantsAsStream() + .sorted(Comparator.comparing(HoodieInstant::requestedTime).reversed()) + .map(instant -> { + try { + HoodieCommitMetadata commitMetadata = + metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantDetails(instant).get(), HoodieCommitMetadata.class); + return Pair.of(instant, commitMetadata); + } catch (IOException e) { + throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); + } + }); + } + + @Override + public Option readCleanerInfoAsBytes(HoodieInstant instant) { + // Cleaner metadata are always stored only in timeline .hoodie + return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); + } + + @Override + public Option readRollbackInfoAsBytes(HoodieInstant instant) { + // Rollback metadata are always stored only in timeline .hoodie + return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); + } + + @Override + public Option readRestoreInfoAsBytes(HoodieInstant instant) { + // Rollback metadata are always stored only in timeline .hoodie + return readDataFromPath(getInstantFileNamePath(getInstantFileName(instant))); + } + + //----------------------------------------------------------------- + // BEGIN - COMPACTION RELATED META-DATA MANAGEMENT. + //----------------------------------------------------------------- + + @Override + public Option readCompactionPlanAsBytes(HoodieInstant instant) { + return readDataFromPath(new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant))); + } + + @Override + public Option readIndexPlanAsBytes(HoodieInstant instant) { + return readDataFromPath(new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant))); + } + + @Override + public HoodieInstant revertInstantFromInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, inflightInstant.getAction(), inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionPendingState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public HoodieInstant revertLogCompactionInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant requestedInstant = + instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, LOG_COMPACTION_ACTION, inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Pass empty data since it is read from the corresponding .aux/.compaction instant file + transitionPendingState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public HoodieInstant transitionCompactionRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, COMPACTION_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflightInstant, Option.empty()); + return inflightInstant; + } + + @Override + public HoodieInstant transitionLogCompactionRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = + instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, LOG_COMPACTION_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflightInstant, Option.empty()); + return inflightInstant; + } + + @Override + public HoodieInstant transitionCompactionInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, + Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, COMMIT_ACTION, inflightInstant.requestedTime()); + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionLogCompactionInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, DELTA_COMMIT_ACTION, inflightInstant.requestedTime()); + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + //----------------------------------------------------------------- + // END - COMPACTION RELATED META-DATA MANAGEMENT + //----------------------------------------------------------------- + + @Override + public HoodieInstant transitionCleanInflightToComplete(boolean shouldLock, HoodieInstant inflightInstant, + Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, CLEAN_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, CLEAN_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflight, data); + return inflight; + } + + @Override + public HoodieInstant transitionRollbackInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, ROLLBACK_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionRollbackRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, ROLLBACK_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflight, Option.empty()); + return inflight; + } + + @Override + public HoodieInstant transitionRestoreRequestedToInflight(HoodieInstant requestedInstant) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.RESTORE_ACTION), "Transition to inflight requested for a restore instant with diff action " + + requestedInstant); + ValidationUtils.checkArgument(requestedInstant.isRequested(), "Transition to inflight requested for an instant not in requested state " + requestedInstant.toString()); + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, RESTORE_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflight, Option.empty()); + return inflight; + } + + @Override + public HoodieInstant transitionReplaceRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, REPLACE_COMMIT_ACTION, requestedInstant.requestedTime()); + // Then write to timeline + transitionPendingState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + @Override + public HoodieInstant transitionClusterRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested()); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, CLUSTERING_ACTION, requestedInstant.requestedTime()); + // Then write to timeline + transitionPendingState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + @Override + public HoodieInstant transitionReplaceInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant transitionClusterInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.requestedTime()); + // Then write to timeline + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + private void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { + transitionPendingState(fromInstant, toInstant, data, false); + } + + protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, + HoodieInstant toInstant, Option data) { + ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); + String fromInstantFileName = instantFileNameGenerator.getFileName(fromInstant); + // Ensures old state exists in timeline + LOG.info("Checking for file exists ?" + getInstantFileNamePath(fromInstantFileName)); + try { + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Re-create the .inflight file by opening a new file and write the commit metadata in + createFileInMetaPath(fromInstantFileName, data, false); + StoragePath fromInstantPath = getInstantFileNamePath(fromInstantFileName); + HoodieInstant instantWithCompletionTime = + instantGenerator.createNewInstant(toInstant.getState(), toInstant.getAction(), + toInstant.requestedTime(), metaClient.createNewInstantTime(false)); + StoragePath toInstantPath = + getInstantFileNamePath(instantFileNameGenerator.getFileName(instantWithCompletionTime)); + boolean success = metaClient.getStorage().rename(fromInstantPath, toInstantPath); + if (!success) { + throw new HoodieIOException( + "Could not rename " + fromInstantPath + " to " + toInstantPath); + } + } else { + ValidationUtils.checkArgument( + metaClient.getStorage().exists(getInstantFileNamePath(fromInstantFileName))); + createCompleteFileInMetaPath(shouldLock, toInstant, data); + } + } catch (IOException e) { + throw new HoodieIOException("Could not complete " + fromInstant, e); + } + } + + protected void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, + boolean allowRedundantTransitions) { + ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime()), String.format("%s and %s are not consistent when transition state.", fromInstant, toInstant)); + String fromInstantFileName = instantFileNameGenerator.getFileName(fromInstant); + String toInstantFileName = instantFileNameGenerator.getFileName(toInstant); + try { + HoodieStorage storage = metaClient.getStorage(); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + // Re-create the .inflight file by opening a new file and write the commit metadata in + createFileInMetaPath(fromInstantFileName, data, allowRedundantTransitions); + StoragePath fromInstantPath = getInstantFileNamePath(fromInstantFileName); + StoragePath toInstantPath = getInstantFileNamePath(toInstantFileName); + boolean success = storage.rename(fromInstantPath, toInstantPath); + if (!success) { + throw new HoodieIOException("Could not rename " + fromInstantPath + " to " + toInstantPath); + } + } else { + // Ensures old state exists in timeline + ValidationUtils.checkArgument(storage.exists(getInstantFileNamePath(fromInstantFileName)), + "File " + getInstantFileNamePath(fromInstantFileName) + " does not exist!"); + // Use Write Once to create Target File + if (allowRedundantTransitions) { + FileIOUtils.createFileInPath(storage, getInstantFileNamePath(toInstantFileName), data); + } else { + storage.createImmutableFileInPath(getInstantFileNamePath(toInstantFileName), data); + } + LOG.info("Create new file for toInstant ?" + getInstantFileNamePath(toInstantFileName)); + } + } catch (IOException e) { + throw new HoodieIOException("Could not complete " + fromInstant, e); + } + } + + protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant inflight) { + ValidationUtils.checkArgument(completed.isCompleted()); + ValidationUtils.checkArgument(inflight.isInflight()); + ValidationUtils.checkArgument(completed.requestedTime().equals(inflight.requestedTime())); + StoragePath inflightFilePath = getInstantFileNamePath(instantFileNameGenerator.getFileName(inflight)); + StoragePath completedFilePath = getInstantFileNamePath(getInstantFileName(completed)); + try { + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + if (!metaClient.getStorage().exists(inflightFilePath)) { + boolean success = metaClient.getStorage().rename(completedFilePath, inflightFilePath); + if (!success) { + throw new HoodieIOException( + "Could not rename " + completedFilePath + " to " + inflightFilePath); + } + } + } else { + StoragePath requestedInstantFilePath = getInstantFileNamePath( + instantFileNameGenerator.getFileName(instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, inflight.getAction(), + inflight.requestedTime()))); + + // If inflight and requested files do not exist, create one + if (!metaClient.getStorage().exists(requestedInstantFilePath)) { + metaClient.getStorage().create(requestedInstantFilePath, false).close(); + } + + if (!metaClient.getStorage().exists(inflightFilePath)) { + metaClient.getStorage().create(inflightFilePath, false).close(); + } + + boolean success = metaClient.getStorage().deleteFile(completedFilePath); + ValidationUtils.checkArgument(success, "State Reverting failed"); + } + } catch (IOException e) { + throw new HoodieIOException("Could not complete revert " + completed, e); + } + } + + private StoragePath getInstantFileNamePath(String fileName) { + return new StoragePath(fileName.contains(SCHEMA_COMMIT_ACTION) ? metaClient.getSchemaFolderName() : metaClient.getMetaPath().toString(), fileName); + } + + public void transitionRequestedToInflight(String commitType, String inFlightInstant) { + HoodieInstant requested = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant); + transitionRequestedToInflight(requested, Option.empty(), false); + } + + public void transitionRequestedToInflight(HoodieInstant requested, Option content) { + transitionRequestedToInflight(requested, content, false); + } + + public void transitionRequestedToInflight(HoodieInstant requested, Option content, + boolean allowRedundantTransitions) { + HoodieInstant inflight = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, requested.getAction(), requested.requestedTime()); + ValidationUtils.checkArgument(requested.isRequested(), "Instant " + requested + " in wrong state"); + transitionPendingState(requested, inflight, content, allowRedundantTransitions); + } + + public void saveToCompactionRequested(HoodieInstant instant, Option content) { + saveToCompactionRequested(instant, content, false); + } + + public void saveToCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.COMPACTION_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, overwrite); + } + + public void saveToLogCompactionRequested(HoodieInstant instant, Option content) { + saveToLogCompactionRequested(instant, content, false); + } + + public void saveToLogCompactionRequested(HoodieInstant instant, Option content, boolean overwrite) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.LOG_COMPACTION_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, overwrite); + } + + @Override + public void saveToPendingReplaceCommit(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToPendingClusterCommit(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToCleanRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.CLEAN_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToRollbackRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public void saveToRestoreRequested(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.RESTORE_ACTION)); + ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); + // Plan is stored in meta path + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + @Override + public HoodieInstant transitionIndexRequestedToInflight(HoodieInstant requestedInstant, Option data) { + ValidationUtils.checkArgument(requestedInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", requestedInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(requestedInstant.isRequested(), + String.format("Instant %s not in requested state", requestedInstant.requestedTime())); + HoodieInstant inflightInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, INDEXING_ACTION, requestedInstant.requestedTime()); + transitionPendingState(requestedInstant, inflightInstant, data); + return inflightInstant; + } + + @Override + public HoodieInstant transitionIndexInflightToComplete(boolean shouldLock, + HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.requestedTime())); + HoodieInstant commitInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, INDEXING_ACTION, inflightInstant.requestedTime()); + transitionStateToComplete(shouldLock, inflightInstant, commitInstant, data); + return commitInstant; + } + + @Override + public HoodieInstant revertIndexInflightToRequested(HoodieInstant inflightInstant) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", inflightInstant.getAction(), INDEXING_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight(), + String.format("Instant %s not inflight", inflightInstant.requestedTime())); + HoodieInstant requestedInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, INDEXING_ACTION, inflightInstant.requestedTime()); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + transitionPendingState(inflightInstant, requestedInstant, Option.empty()); + } else { + deleteInflight(inflightInstant); + } + return requestedInstant; + } + + @Override + public void saveToPendingIndexAction(HoodieInstant instant, Option content) { + ValidationUtils.checkArgument(instant.getAction().equals(HoodieTimeline.INDEXING_ACTION), + String.format("%s is not equal to %s action", instant.getAction(), INDEXING_ACTION)); + createFileInMetaPath(instantFileNameGenerator.getFileName(instant), content, false); + } + + protected void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { + StoragePath fullPath = getInstantFileNamePath(filename); + if (allowOverwrite || metaClient.getTimelineLayoutVersion().isNullVersion()) { + FileIOUtils.createFileInPath(metaClient.getStorage(), fullPath, content); + } else { + metaClient.getStorage().createImmutableFileInPath(fullPath, content); + } + } + + protected void createCompleteFileInMetaPath(boolean shouldLock, HoodieInstant instant, Option content) { + TimeGenerator timeGenerator = TimeGenerators + .getTimeGenerator(metaClient.getTimeGeneratorConfig(), metaClient.getStorageConf()); + timeGenerator.consumeTimestamp(!shouldLock, currentTimeMillis -> { + String completionTime = HoodieInstantTimeGenerator.formatDate(new Date(currentTimeMillis)); + String fileName = instantFileNameGenerator.getFileName(completionTime, instant); + StoragePath fullPath = getInstantFileNamePath(fileName); + if (metaClient.getTimelineLayoutVersion().isNullVersion()) { + FileIOUtils.createFileInPath(metaClient.getStorage(), fullPath, content); + } else { + metaClient.getStorage().createImmutableFileInPath(fullPath, content); + } + LOG.info("Created new file for toInstant ?" + fullPath); + }); + } + + protected Option readDataFromPath(StoragePath detailPath) { + try (InputStream is = metaClient.getStorage().open(detailPath)) { + return Option.of(FileIOUtils.readAsByteArray(is)); + } catch (IOException e) { + throw new HoodieIOException("Could not read commit details from " + detailPath, e); + } + } + + public HoodieActiveTimeline reload() { + return new ActiveTimelineV2(metaClient); + } + + public void copyInstant(HoodieInstant instant, StoragePath dstDir) { + StoragePath srcPath = new StoragePath(metaClient.getMetaPath(), getInstantFileName(instant)); + StoragePath dstPath = new StoragePath(dstDir, getInstantFileName(instant)); + try { + HoodieStorage storage = metaClient.getStorage(); + storage.createDirectory(dstDir); + FileIOUtils.copy(storage, srcPath, storage, dstPath, false, true); + } catch (IOException e) { + throw new HoodieIOException("Could not copy instant from " + srcPath + " to " + dstPath, e); + } + } + + @Override + public Set getValidExtensions() { + return VALID_EXTENSIONS_IN_ACTIVE_TIMELINE; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineLoaderV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineLoaderV2.java new file mode 100644 index 000000000000..0e7db244b8d9 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineLoaderV2.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.LSMTimeline; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieAvroFileReader; +import org.apache.hudi.io.storage.HoodieIOFactory; +import org.apache.hudi.storage.StoragePath; + +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.jetbrains.annotations.Nullable; + +import java.io.IOException; +import java.util.List; +import java.util.function.BiConsumer; +import java.util.function.Function; + +import static org.apache.hudi.common.util.ConfigUtils.DEFAULT_HUDI_CONFIG_FOR_READER; + +public class ArchivedTimelineLoaderV2 implements ArchivedTimelineLoader { + + private static final String INSTANT_TIME_ARCHIVED_META_FIELD = "instantTime"; + + @Override + public void loadInstants(HoodieTableMetaClient metaClient, + @Nullable HoodieArchivedTimeline.TimeRangeFilter filter, + HoodieArchivedTimeline.LoadMode loadMode, + Function commitsFilter, + BiConsumer recordConsumer) { + try { + // List all files + List fileNames = LSMTimeline.latestSnapshotManifest(metaClient).getFileNames(); + + Schema readSchema = LSMTimeline.getReadSchema(loadMode); + fileNames.stream() + .filter(fileName -> filter == null || LSMTimeline.isFileInRange(filter, fileName)) + .parallel().forEach(fileName -> { + // Read the archived file + try (HoodieAvroFileReader reader = (HoodieAvroFileReader) HoodieIOFactory.getIOFactory(metaClient.getStorage()) + .getReaderFactory(HoodieRecord.HoodieRecordType.AVRO) + .getFileReader(DEFAULT_HUDI_CONFIG_FOR_READER, new StoragePath(metaClient.getArchivePath(), fileName))) { + try (ClosableIterator iterator = reader.getIndexedRecordIterator(HoodieLSMTimelineInstant.getClassSchema(), readSchema)) { + while (iterator.hasNext()) { + GenericRecord record = (GenericRecord) iterator.next(); + String instantTime = record.get(INSTANT_TIME_ARCHIVED_META_FIELD).toString(); + if ((filter == null || filter.isInRange(instantTime)) + && commitsFilter.apply(record)) { + recordConsumer.accept(instantTime, record); + } + } + } + } catch (IOException ioException) { + throw new HoodieIOException("Error open file reader for path: " + + new StoragePath(metaClient.getArchivePath(), fileName)); + } + }); + } catch (IOException e) { + throw new HoodieIOException( + "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineV2.java new file mode 100644 index 000000000000..c388722e9e30 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/ArchivedTimelineV2.java @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.generic.GenericRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nullable; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ConcurrentHashMap; +import java.util.function.BiConsumer; +import java.util.function.Function; + +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; + +public class ArchivedTimelineV2 extends BaseTimelineV2 implements HoodieArchivedTimeline { + private static final String INSTANT_TIME_ARCHIVED_META_FIELD = "instantTime"; + public static final String COMPLETION_TIME_ARCHIVED_META_FIELD = "completionTime"; + private static final String ACTION_ARCHIVED_META_FIELD = "action"; + private static final String METADATA_ARCHIVED_META_FIELD = "metadata"; + private static final String PLAN_ARCHIVED_META_FIELD = "plan"; + private HoodieTableMetaClient metaClient; + private final Map readCommits = new ConcurrentHashMap<>(); + + private static final Logger LOG = LoggerFactory.getLogger(HoodieArchivedTimeline.class); + + /** + * Used for loading the archived timeline incrementally, the earliest loaded instant time get memorized + * each time the timeline is loaded. The instant time is then used as the end boundary + * of the next loading. + */ + private String cursorInstant; + private final ArchivedTimelineLoader timelineLoader = new ArchivedTimelineLoaderV2(); + + /** + * Loads all the archived instants. + * Note that there is no lazy loading, so this may not work if the archived timeline range is really long. + * TBD: Should we enforce maximum time range? + */ + public ArchivedTimelineV2(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + setInstants(this.loadInstants()); + this.cursorInstant = firstInstant().map(HoodieInstant::requestedTime).orElse(null); + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + } + + /** + * Loads completed instants from startTs(inclusive). + * Note that there is no lazy loading, so this may not work if really early startTs is specified. + */ + public ArchivedTimelineV2(HoodieTableMetaClient metaClient, String startTs) { + this.metaClient = metaClient; + setInstants(loadInstants(new HoodieArchivedTimeline.StartTsFilter(startTs), HoodieArchivedTimeline.LoadMode.METADATA)); + this.cursorInstant = startTs; + // multiple casts will make this lambda serializable - + // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 + this.details = (Function> & Serializable) this::getInstantDetails; + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + public ArchivedTimelineV2() { + } + + /** + * This method is only used when this object is deserialized in a spark executor. + * + * @deprecated + */ + private void readObject(java.io.ObjectInputStream in) throws IOException, ClassNotFoundException { + in.defaultReadObject(); + } + + public void loadInstantDetailsInMemory(String startTs, String endTs) { + loadInstants(startTs, endTs); + } + + public void loadCompletedInstantDetailsInMemory() { + loadInstants(null, HoodieArchivedTimeline.LoadMode.METADATA); + } + + public void loadCompactionDetailsInMemory(String compactionInstantTime) { + loadCompactionDetailsInMemory(compactionInstantTime, compactionInstantTime); + } + + public void loadCompactionDetailsInMemory(String startTs, String endTs) { + // load compactionPlan + loadInstants(new HoodieArchivedTimeline.TimeRangeFilter(startTs, endTs), HoodieArchivedTimeline.LoadMode.PLAN, + record -> record.get(ACTION_ARCHIVED_META_FIELD).toString().equals(COMMIT_ACTION) + && record.get(PLAN_ARCHIVED_META_FIELD) != null + ); + } + + @Override + public void clearInstantDetailsFromMemory(String instantTime) { + this.readCommits.remove(instantTime); + } + + @Override + public void clearInstantDetailsFromMemory(String startTs, String endTs) { + this.findInstantsInRange(startTs, endTs).getInstants().forEach(instant -> + this.readCommits.remove(instant.requestedTime())); + } + + @Override + public Option getInstantDetails(HoodieInstant instant) { + return Option.ofNullable(readCommits.get(instant.requestedTime())); + } + + @Override + public HoodieArchivedTimeline reload() { + return new ArchivedTimelineV2(metaClient); + } + + @Override + public HoodieArchivedTimeline reload(String startTs) { + if (this.cursorInstant != null) { + if (InstantComparison.compareTimestamps(startTs, LESSER_THAN, this.cursorInstant)) { + appendInstants(loadInstants(new HoodieArchivedTimeline.ClosedOpenTimeRangeFilter(startTs, this.cursorInstant), HoodieArchivedTimeline.LoadMode.METADATA)); + this.cursorInstant = startTs; + } + return this; + } else { + // a null cursor instant indicates an empty timeline + return new ArchivedTimelineV2(metaClient, startTs); + } + } + + private HoodieInstant readCommit(String instantTime, GenericRecord record, Option> instantDetailsConsumer) { + final String action = record.get(ACTION_ARCHIVED_META_FIELD).toString(); + final String completionTime = record.get(COMPLETION_TIME_ARCHIVED_META_FIELD).toString(); + instantDetailsConsumer.ifPresent(consumer -> consumer.accept(instantTime, record)); + return instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, action, instantTime, completionTime); + } + + @Nullable + private BiConsumer getInstantDetailsFunc(HoodieArchivedTimeline.LoadMode loadMode) { + switch (loadMode) { + case METADATA: + return (instant, record) -> { + ByteBuffer commitMeta = (ByteBuffer) record.get(METADATA_ARCHIVED_META_FIELD); + if (commitMeta != null) { + // in case the entry comes from an empty completed meta file + this.readCommits.put(instant, commitMeta.array()); + } + }; + case PLAN: + return (instant, record) -> { + ByteBuffer plan = (ByteBuffer) record.get(PLAN_ARCHIVED_META_FIELD); + if (plan != null) { + // in case the entry comes from an empty completed meta file + this.readCommits.put(instant, plan.array()); + } + }; + default: + return null; + } + } + + private List loadInstants() { + return loadInstants(null, HoodieArchivedTimeline.LoadMode.ACTION); + } + + private List loadInstants(String startTs, String endTs) { + return loadInstants(new HoodieArchivedTimeline.TimeRangeFilter(startTs, endTs), HoodieArchivedTimeline.LoadMode.METADATA); + } + + private List loadInstants(HoodieArchivedTimeline.TimeRangeFilter filter, HoodieArchivedTimeline.LoadMode loadMode) { + return loadInstants(filter, loadMode, r -> true); + } + + /** + * This is method to read selected instants. Do NOT use this directly use one of the helper methods above + * If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details + * If filter is specified, only the filtered instants are loaded + * If commitsFilter is specified, only the filtered records are loaded. + */ + private List loadInstants( + @Nullable HoodieArchivedTimeline.TimeRangeFilter filter, + HoodieArchivedTimeline.LoadMode loadMode, + Function commitsFilter) { + Map instantsInRange = new ConcurrentHashMap<>(); + Option> instantDetailsConsumer = Option.ofNullable(getInstantDetailsFunc(loadMode)); + timelineLoader.loadInstants(metaClient, filter, loadMode, commitsFilter, + (instantTime, avroRecord) -> instantsInRange.putIfAbsent(instantTime, readCommit(instantTime, avroRecord, instantDetailsConsumer))); + List result = new ArrayList<>(instantsInRange.values()); + Collections.sort(result); + return result; + } + + @Override + public HoodieTimeline getWriteTimeline() { + // filter in-memory instants + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, + LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION); + return new BaseTimelineV2(getInstantsAsStream().filter(i -> + readCommits.containsKey(i.requestedTime())) + .filter(s -> validActions.contains(s.getAction())), details); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/BaseTimelineV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/BaseTimelineV2.java new file mode 100644 index 000000000000..424969744076 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/BaseTimelineV2.java @@ -0,0 +1,154 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.table.timeline.BaseHoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.ClusteringUtils; +import org.apache.hudi.common.util.CollectionUtils; +import org.apache.hudi.common.util.Option; + +import java.util.List; +import java.util.Set; +import java.util.function.Function; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +public class BaseTimelineV2 extends BaseHoodieTimeline { + + public BaseTimelineV2(Stream instants, Function> details) { + this(instants, details, TimelineLayout.fromVersion(TimelineLayoutVersion.LAYOUT_VERSION_2)); + } + + public BaseTimelineV2(Stream instants, Function> details, TimelineLayout layout) { + super(instants, details, layout.getTimelineFactory(), layout.getInstantComparator(), layout.getInstantGenerator()); + } + + /** + * For serialization and de-serialization only. + * + * @deprecated + */ + @Deprecated + public BaseTimelineV2() { + super(TimelineLayout.fromVersion(TimelineLayoutVersion.LAYOUT_VERSION_2)); + } + + @Override + public HoodieTimeline getWriteTimeline() { + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION); + return factory.createDefaultTimeline(getInstantsAsStream().filter(s -> validActions.contains(s.getAction())), details); + } + + @Override + public TimelineLayoutVersion getTimelineLayoutVersion() { + return TimelineLayoutVersion.LAYOUT_VERSION_2; + } + + @Override + public HoodieTimeline filterPendingClusteringTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) && !s.isCompleted()), details); + } + + @Override + public HoodieTimeline filterPendingReplaceOrClusteringTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> (s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) || s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) + && !s.isCompleted()), details); + } + + @Override + public HoodieTimeline filterPendingReplaceClusteringAndCompactionTimeline() { + return factory.createDefaultTimeline(getInstantsAsStream().filter( + s -> !s.isCompleted() && (s.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) + || s.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) + || s.getAction().equals(HoodieTimeline.COMPACTION_ACTION))), details); + } + + @Override + public HoodieTimeline getCommitsTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION)); + } + + @Override + public HoodieTimeline getCommitAndReplaceTimeline() { + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION, CLUSTERING_ACTION)); + } + + @Override + public Option getLastClusteringInstant() { + return Option.fromJavaOptional(getCommitsTimeline().filter(s -> s.getAction().equalsIgnoreCase(HoodieTimeline.REPLACE_COMMIT_ACTION) + || s.getAction().equalsIgnoreCase(HoodieTimeline.CLUSTERING_ACTION)) + .getReverseOrderedInstants() + .filter(i -> ClusteringUtils.isClusteringInstant(this, i, instantGenerator)) + .findFirst()); + } + + @Override + public Option getFirstPendingClusterInstant() { + return getLastOrFirstPendingClusterInstant(false); + } + + @Override + public Option getLastPendingClusterInstant() { + return getLastOrFirstPendingClusterInstant(true); + } + + private Option getLastOrFirstPendingClusterInstant(boolean isLast) { + HoodieTimeline pendingClusteringTimeline = filterPendingReplaceOrClusteringTimeline(); + Stream clusterStream; + if (isLast) { + clusterStream = pendingClusteringTimeline.getReverseOrderedInstants(); + } else { + clusterStream = pendingClusteringTimeline.getInstantsAsStream(); + } + return Option.fromJavaOptional(clusterStream + .filter(i -> ClusteringUtils.isClusteringInstant(this, i, instantGenerator)).findFirst()); + } + + @Override + public boolean isPendingClusteringInstant(String instantTime) { + return getOrCreatePendingClusteringInstantSet().contains(instantTime); + } + + private Set getOrCreatePendingClusteringInstantSet() { + if (this.pendingClusteringInstants == null) { + synchronized (this) { + if (this.pendingClusteringInstants == null) { + List pendingClusterInstants = getCommitsTimeline().filterPendingReplaceOrClusteringTimeline().getInstants(); + // Validate that there are no instants with same timestamp + pendingClusterInstants.stream().collect(Collectors.groupingBy(HoodieInstant::requestedTime)).forEach((timestamp, instants) -> { + if (instants.size() > 1) { + throw new IllegalStateException("Multiple instants with same timestamp: " + timestamp + " instants: " + instants); + } + }); + // Filter replace commits down to those that are due to clustering + this.pendingClusteringInstants = pendingClusterInstants.stream() + .filter(instant -> ClusteringUtils.isClusteringInstant(this, instant, instantGenerator)) + .map(HoodieInstant::requestedTime).collect(Collectors.toSet()); + } + } + } + return this.pendingClusteringInstants; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CommitMetadataSerDeV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CommitMetadataSerDeV2.java new file mode 100644 index 000000000000..5d5b5c29384e --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CommitMetadataSerDeV2.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.avro.model.HoodieCommitMetadata; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.MetadataConversionUtils; +import org.apache.hudi.common.util.JsonUtils; +import org.apache.hudi.common.util.Option; + +import org.apache.avro.file.DataFileWriter; +import org.apache.avro.io.DatumWriter; +import org.apache.avro.specific.SpecificDatumWriter; +import org.apache.avro.specific.SpecificRecordBase; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertCommitMetadataToJsonBytes; +import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeCommitMetadata; +import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes; + +public class CommitMetadataSerDeV2 implements CommitMetadataSerDe { + + @Override + public T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException { + try { + if (bytes.length == 0) { + return clazz.newInstance(); + } + if (instant.isLegacy()) { + try { + return fromJsonString(fromUTF8Bytes(bytes), clazz); + } catch (Exception e) { + throw new IOException("unable to read legacy commit metadata for instant " + instant, e); + } + } + return fromJsonString( + fromUTF8Bytes( + convertCommitMetadataToJsonBytes(deserializeCommitMetadata(bytes), org.apache.hudi.avro.model.HoodieCommitMetadata.class)), + clazz); + } catch (Exception e) { + throw new IOException("unable to read commit metadata for instant " + instant + " bytes length: " + bytes.length, e); + } + } + + public static T fromJsonString(String jsonStr, Class clazz) throws Exception { + if (jsonStr == null || jsonStr.isEmpty()) { + // For empty commit file + return clazz.newInstance(); + } + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); + } + + @Override + public Option serialize(org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata) throws IOException { + return serializeAvroMetadata(MetadataConversionUtils.convertCommitMetadata(commitMetadata), HoodieCommitMetadata.class); + } + + public static Option serializeAvroMetadata(T metadata, Class clazz) + throws IOException { + DatumWriter datumWriter = new SpecificDatumWriter<>(clazz); + DataFileWriter fileWriter = new DataFileWriter<>(datumWriter); + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + fileWriter.create(metadata.getSchema(), baos); + fileWriter.append(metadata); + fileWriter.flush(); + return Option.of(baos.toByteArray()); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java new file mode 100644 index 000000000000..939613092e1f --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/CompletionTimeQueryViewV2.java @@ -0,0 +1,348 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.InstantRange; +import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.VisibleForTesting; + +import org.apache.avro.generic.GenericRecord; + +import java.io.Serializable; +import java.time.Instant; +import java.util.Collections; +import java.util.Comparator; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; +import java.util.function.Function; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.table.read.IncrementalQueryAnalyzer.START_COMMIT_EARLIEST; +import static org.apache.hudi.common.table.timeline.HoodieArchivedTimeline.COMPLETION_TIME_ARCHIVED_META_FIELD; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; + +/** + * Query view for instant completion time. + */ +public class CompletionTimeQueryViewV2 implements CompletionTimeQueryView, Serializable { + private static final long serialVersionUID = 1L; + + private static final long MILLI_SECONDS_IN_THREE_DAYS = 3 * 24 * 3600 * 1000; + + private static final long MILLI_SECONDS_IN_ONE_DAY = 24 * 3600 * 1000; + + private static final Function GET_INSTANT_ONE_DAY_BEFORE = instant -> + HoodieInstantTimeGenerator.instantTimeMinusMillis(instant, MILLI_SECONDS_IN_ONE_DAY); + + private final HoodieTableMetaClient metaClient; + + /** + * Mapping from instant time -> completion time. + * Should be thread-safe data structure. + */ + private final ConcurrentMap instantTimeToCompletionTimeMap; + + /** + * The cursor instant time to eagerly load from, by default load last N days of completed instants. + * It can grow dynamically with lazy loading. e.g. assuming an initial cursor instant as t10, + * a completion query for t5 would trigger lazy loading with this cursor instant updated to t5. + * This sliding window model amortizes redundant loading from different queries. + */ + private volatile String cursorInstant; + + /** + * The first write instant on the active timeline, used for query optimization. + */ + private final String firstNonSavepointCommit; + + /** + * The constructor. + * + * @param metaClient The table meta client. + */ + public CompletionTimeQueryViewV2(HoodieTableMetaClient metaClient) { + this(metaClient, HoodieInstantTimeGenerator.formatDate(new Date(Instant.now().minusMillis(MILLI_SECONDS_IN_THREE_DAYS).toEpochMilli()))); + } + + /** + * The constructor. + * + * @param metaClient The table meta client. + * @param eagerLoadInstant The earliest instant time to eagerly load from, by default load last N days of completed instants. + */ + public CompletionTimeQueryViewV2(HoodieTableMetaClient metaClient, String eagerLoadInstant) { + this.metaClient = metaClient; + this.instantTimeToCompletionTimeMap = new ConcurrentHashMap<>(); + this.cursorInstant = InstantComparison.minInstant(eagerLoadInstant, metaClient.getActiveTimeline().firstInstant().map(HoodieInstant::requestedTime).orElse("")); + // Note: use getWriteTimeline() to keep sync with the fs view visibleCommitsAndCompactionTimeline, see AbstractTableFileSystemView.refreshTimeline. + this.firstNonSavepointCommit = metaClient.getActiveTimeline().getWriteTimeline().getFirstNonSavepointCommit().map(HoodieInstant::requestedTime).orElse(""); + load(); + } + + /** + * Returns whether the instant is completed. + */ + public boolean isCompleted(String instantTime) { + // archival does not proceed beyond the first savepoint, so any instant before that is completed. + return this.instantTimeToCompletionTimeMap.containsKey(instantTime) || isArchived(instantTime); + } + + /** + * Returns whether the instant is archived. + */ + public boolean isArchived(String instantTime) { + return InstantComparison.compareTimestamps(instantTime, LESSER_THAN, this.firstNonSavepointCommit); + } + + /** + * Returns whether the give instant time {@code instantTime} completed before the base instant {@code baseInstant}. + */ + public boolean isCompletedBefore(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); + if (completionTimeOpt.isPresent()) { + return InstantComparison.compareTimestamps(completionTimeOpt.get(), LESSER_THAN, baseInstant); + } + return false; + } + + /** + * Returns whether the given instant time {@code instantTime} is sliced after or on the base instant {@code baseInstant}. + */ + public boolean isSlicedAfterOrOn(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(baseInstant, instantTime); + if (completionTimeOpt.isPresent()) { + return InstantComparison.compareTimestamps(completionTimeOpt.get(), GREATER_THAN_OR_EQUALS, baseInstant); + } + return true; + } + + /** + * Get completion time with a base instant time as a reference to fix the compatibility. + * + * @param baseInstant The base instant + * @param instantTime The instant time to query the completion time with + * @return Probability fixed completion time. + */ + public Option getCompletionTime(String baseInstant, String instantTime) { + Option completionTimeOpt = getCompletionTime(instantTime); + if (completionTimeOpt.isPresent()) { + String completionTime = completionTimeOpt.get(); + if (completionTime.length() != baseInstant.length()) { + // ============================================================== + // LEGACY CODE + // ============================================================== + // Fixes the completion time to reflect the completion sequence correctly + // if the file slice base instant time is not in datetime format. + // For example, many test cases just use integer string as the instant time. + // CAUTION: this fix only works for OCC(Optimistic Concurrency Control). + // for NB-CC(Non-blocking Concurrency Control), the file slicing may be incorrect. + return Option.of(instantTime); + } + } + return completionTimeOpt; + } + + /** + * Queries the completion time with given instant time. + * + * @param instantTime The instant time. + * @return The completion time if the instant finished or empty if it is still pending. + */ + public Option getCompletionTime(String instantTime) { + String completionTime = this.instantTimeToCompletionTimeMap.get(instantTime); + if (completionTime != null) { + return Option.of(completionTime); + } + if (InstantComparison.compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, this.cursorInstant)) { + // the instant is still pending + return Option.empty(); + } + loadCompletionTimeIncrementally(instantTime); + return Option.ofNullable(this.instantTimeToCompletionTimeMap.get(instantTime)); + } + + @Override + + public List getInstantTimes( + HoodieTimeline timeline, + Option startCompletionTime, + Option endCompletionTime, + InstantRange.RangeType rangeType) { + // assumes any instant/transaction lasts at most 1 day to optimize the query efficiency. + return getInstantTimes( + timeline, startCompletionTime, endCompletionTime, rangeType, GET_INSTANT_ONE_DAY_BEFORE); + } + + @VisibleForTesting + public List getInstantTimes( + String startCompletionTime, + String endCompletionTime, + Function earliestInstantTimeFunc) { + return getInstantTimes( + metaClient.getCommitsTimeline().filterCompletedInstants(), + Option.ofNullable(startCompletionTime), + Option.ofNullable(endCompletionTime), + InstantRange.RangeType.CLOSED_CLOSED, + earliestInstantTimeFunc); + } + + /** + * Queries the instant times with given completion time range. + * + * @param timeline The timeline. + * @param startCompletionTime The start completion time of the query range. + * @param endCompletionTime The end completion time of the query range. + * @param rangeType The range type. + * @param earliestInstantTimeFunc The function to generate the earliest instant time boundary + * with the minimum completion time. + * @return The sorted instant time list. + */ + private List getInstantTimes( + HoodieTimeline timeline, + Option startCompletionTime, + Option endCompletionTime, + InstantRange.RangeType rangeType, + Function earliestInstantTimeFunc) { + final boolean startFromEarliest = START_COMMIT_EARLIEST.equalsIgnoreCase(startCompletionTime.orElse(null)); + String earliestInstantToLoad = null; + if (startCompletionTime.isPresent() && !startFromEarliest) { + earliestInstantToLoad = earliestInstantTimeFunc.apply(startCompletionTime.get()); + } else if (endCompletionTime.isPresent()) { + earliestInstantToLoad = earliestInstantTimeFunc.apply(endCompletionTime.get()); + } + + // ensure the earliest instant boundary be loaded. + if (earliestInstantToLoad != null && InstantComparison.compareTimestamps(this.cursorInstant, GREATER_THAN, earliestInstantToLoad)) { + loadCompletionTimeIncrementally(earliestInstantToLoad); + } + + if (startCompletionTime.isEmpty() && endCompletionTime.isPresent()) { + // returns the last instant that finished at or before the given completion time 'endTime'. + String maxInstantTime = timeline.getInstantsAsStream() + .filter(instant -> instant.isCompleted() && InstantComparison.compareTimestamps(instant.getCompletionTime(), LESSER_THAN_OR_EQUALS, endCompletionTime.get())) + .max(Comparator.comparing(HoodieInstant::getCompletionTime)).map(HoodieInstant::requestedTime).orElse(null); + if (maxInstantTime != null) { + return Collections.singletonList(maxInstantTime); + } + // fallback to archived timeline + return this.instantTimeToCompletionTimeMap.entrySet().stream() + .filter(entry -> InstantComparison.compareTimestamps(entry.getValue(), LESSER_THAN_OR_EQUALS, endCompletionTime.get())) + .map(Map.Entry::getKey).collect(Collectors.toList()); + } + + if (startFromEarliest) { + // expedience for snapshot read: ['earliest', _) to avoid loading unnecessary instants. + startCompletionTime = Option.empty(); + } + + if (startCompletionTime.isEmpty() && endCompletionTime.isEmpty()) { + // (_, _): read the latest snapshot. + return timeline.filterCompletedInstants().lastInstant().map(instant -> Collections.singletonList(instant.requestedTime())).orElse(Collections.emptyList()); + } + + final InstantRange instantRange = InstantRange.builder() + .rangeType(rangeType) + .startInstant(startCompletionTime.orElse(null)) + .endInstant(endCompletionTime.orElse(null)) + .nullableBoundary(true) + .build(); + return this.instantTimeToCompletionTimeMap.entrySet().stream() + .filter(entry -> instantRange.isInRange(entry.getValue())) + .map(Map.Entry::getKey).sorted().collect(Collectors.toList()); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + private void loadCompletionTimeIncrementally(String startTime) { + // the 'startTime' should be out of the eager loading range, switch to a lazy loading. + // This operation is resource costly. + synchronized (this) { + if (InstantComparison.compareTimestamps(startTime, LESSER_THAN, this.cursorInstant)) { + metaClient.getTimelineLayout().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, + new HoodieArchivedTimeline.ClosedOpenTimeRangeFilter(startTime, this.cursorInstant), + HoodieArchivedTimeline.LoadMode.TIME, + r -> true, + this::readCompletionTime); + } + // refresh the start instant + this.cursorInstant = startTime; + } + } + + /** + * This is method to read instant completion time. + * This would also update 'startToCompletionInstantTimeMap' map with start time/completion time pairs. + * Only instants starts from 'startInstant' (inclusive) are considered. + */ + private void load() { + // load active instants first. + this.metaClient.getActiveTimeline() + .filterCompletedInstants().getInstantsAsStream() + .forEach(instant -> setCompletionTime(instant.requestedTime(), instant.getCompletionTime())); + // then load the archived instants. + metaClient.getTimelineLayout().getTimelineFactory().createArchivedTimelineLoader().loadInstants(metaClient, + new HoodieArchivedTimeline.StartTsFilter(this.cursorInstant), + HoodieArchivedTimeline.LoadMode.TIME, + r -> true, + this::readCompletionTime); + } + + private void readCompletionTime(String instantTime, GenericRecord record) { + final String completionTime = record.get(COMPLETION_TIME_ARCHIVED_META_FIELD).toString(); + setCompletionTime(instantTime, completionTime); + } + + private void setCompletionTime(String beginInstantTime, String completionTime) { + if (completionTime == null) { + // the meta-server instant does not have completion time + completionTime = beginInstantTime; + } + this.instantTimeToCompletionTimeMap.putIfAbsent(beginInstantTime, completionTime); + } + + @Override + public String getCursorInstant() { + return cursorInstant; + } + + @Override + public boolean isEmptyTable() { + return this.instantTimeToCompletionTimeMap.isEmpty(); + } + + @Override + public void close() { + this.instantTimeToCompletionTimeMap.clear(); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantComparatorV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantComparatorV2.java new file mode 100644 index 000000000000..4b20708789be --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantComparatorV2.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantComparator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.common.InstantComparators; + +import java.io.Serializable; +import java.util.Comparator; +import java.util.HashMap; +import java.util.Map; + +public class InstantComparatorV2 implements Serializable, InstantComparator { + + /** + * A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants + * for state transitions, this needs to be taken into account. + */ + private static final Map COMPARABLE_ACTIONS = createComparableActionsMap(); + public static final Comparator ACTION_COMPARATOR = + new InstantComparators.ActionComparator(COMPARABLE_ACTIONS); + + public static final Comparator REQUESTED_TIME_BASED_COMPARATOR = + new InstantComparators.RequestedTimeBasedComparator(COMPARABLE_ACTIONS); + + public static final Comparator COMPLETION_TIME_BASED_COMPARATOR = + new InstantComparators.CompletionTimeBasedComparator(COMPARABLE_ACTIONS); + + public static String getComparableAction(String action) { + return COMPARABLE_ACTIONS.getOrDefault(action, action); + } + + private static Map createComparableActionsMap() { + Map comparableMap = new HashMap<>(); + comparableMap.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION); + comparableMap.put(HoodieTimeline.LOG_COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION); + comparableMap.put(HoodieTimeline.CLUSTERING_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION); + return comparableMap; + } + + @Override + public Comparator actionOnlyComparator() { + return ACTION_COMPARATOR; + } + + @Override + public Comparator requestedTimeOrderedComparator() { + return REQUESTED_TIME_BASED_COMPARATOR; + } + + @Override + public Comparator completionTimeOrderedComparator() { + return COMPLETION_TIME_BASED_COMPARATOR; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameGeneratorV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameGeneratorV2.java new file mode 100644 index 000000000000..86b1555de01b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameGeneratorV2.java @@ -0,0 +1,323 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.ValidationUtils; + +public class InstantFileNameGeneratorV2 implements InstantFileNameGenerator { + + @Override + public String makeCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.COMMIT_EXTENSION); + } + + @Override + public String makeInflightCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedCommitFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMMIT_EXTENSION); + } + + @Override + public String makeCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.CLEAN_EXTENSION); + } + + @Override + public String makeRequestedCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLEAN_EXTENSION); + } + + @Override + public String makeInflightCleanerFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLEAN_EXTENSION); + } + + @Override + public String makeRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.ROLLBACK_EXTENSION); + } + + @Override + public String makeRequestedRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_ROLLBACK_EXTENSION); + } + + @Override + public String makeRequestedRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_RESTORE_EXTENSION); + } + + @Override + public String makeInflightRollbackFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_ROLLBACK_EXTENSION); + } + + @Override + public String makeInflightSavePointFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); + } + + @Override + public String makeSavePointFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.SAVEPOINT_EXTENSION); + } + + @Override + public String makeInflightDeltaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedDeltaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_DELTA_COMMIT_EXTENSION); + } + + @Override + public String makeInflightCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); + } + + @Override + public String makeRequestedCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); + } + + @Override + public String makeInflightLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_LOG_COMPACTION_EXTENSION); + } + + @Override + public String makeRequestedLogCompactionFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_LOG_COMPACTION_EXTENSION); + } + + @Override + public String makeRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.RESTORE_EXTENSION); + } + + @Override + public String makeInflightRestoreFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION); + } + + @Override + public String makeReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeInflightReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedClusteringFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_CLUSTERING_COMMIT_EXTENSION); + } + + @Override + public String makeInflightClusteringFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_CLUSTERING_COMMIT_EXTENSION); + } + + @Override + public String makeDeltaFileName(String instantTime) { + return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION; + } + + @Override + public String getCommitFromCommitFile(String commitFileName) { + return commitFileName.split("\\.")[0]; + } + + @Override + public String makeFileNameAsComplete(String fileName) { + return fileName.replace(HoodieTimeline.INFLIGHT_EXTENSION, ""); + } + + @Override + public String makeFileNameAsInflight(String fileName) { + return StringUtils.join(fileName, HoodieTimeline.INFLIGHT_EXTENSION); + } + + @Override + public String makeIndexCommitFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeInflightIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeRequestedIndexFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_INDEX_COMMIT_EXTENSION); + } + + @Override + public String makeSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.SAVE_SCHEMA_ACTION_EXTENSION); + } + + @Override + public String makeInflightSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.INFLIGHT_SAVE_SCHEMA_ACTION_EXTENSION); + } + + @Override + public String makeRequestSchemaFileName(String instantTime) { + return StringUtils.join(instantTime, HoodieTimeline.REQUESTED_SAVE_SCHEMA_ACTION_EXTENSION); + } + + private String getPendingFileName(HoodieInstant instant) { + String action = instant.getAction(); + String timestamp = instant.requestedTime(); + if (HoodieTimeline.COMMIT_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightCommitFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedCommitFileName(timestamp); + } + } else if (HoodieTimeline.CLEAN_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightCleanerFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedCleanerFileName(timestamp); + } + } else if (HoodieTimeline.ROLLBACK_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightRollbackFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedRollbackFileName(timestamp); + } + } else if (HoodieTimeline.SAVEPOINT_ACTION.equals(action)) { + return makeInflightSavePointFileName(timestamp); + } else if (HoodieTimeline.DELTA_COMMIT_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightDeltaFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedDeltaFileName(timestamp); + } + } else if (HoodieTimeline.COMPACTION_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightCompactionFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedCompactionFileName(timestamp); + } + } else if (HoodieTimeline.LOG_COMPACTION_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightLogCompactionFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedLogCompactionFileName(timestamp); + } + } else if (HoodieTimeline.RESTORE_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightRestoreFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedRestoreFileName(timestamp); + } + } else if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightReplaceFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedReplaceFileName(timestamp); + } + } else if (HoodieTimeline.CLUSTERING_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightClusteringFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedClusteringFileName(timestamp); + } + } else if (HoodieTimeline.INDEXING_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightIndexFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestedIndexFileName(timestamp); + } + } else if (HoodieTimeline.SCHEMA_COMMIT_ACTION.equals(action)) { + if (instant.isInflight()) { + return makeInflightSchemaFileName(timestamp); + } else if (instant.isRequested()) { + return makeRequestSchemaFileName(timestamp); + } + } + throw new IllegalArgumentException("Cannot get file name for unknown action " + action); + } + + private String getCompleteFileName(HoodieInstant instant, String completionTime) { + ValidationUtils.checkArgument(!StringUtils.isNullOrEmpty(completionTime), "Completion time should not be empty"); + String timestampWithCompletionTime = instant.isLegacy() ? instant.requestedTime() : instant.requestedTime() + "_" + completionTime; + switch (instant.getAction()) { + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.COMPACTION_ACTION: + return makeCommitFileName(timestampWithCompletionTime); + case HoodieTimeline.CLEAN_ACTION: + return makeCleanerFileName(timestampWithCompletionTime); + case HoodieTimeline.ROLLBACK_ACTION: + return makeRollbackFileName(timestampWithCompletionTime); + case HoodieTimeline.SAVEPOINT_ACTION: + return makeSavePointFileName(timestampWithCompletionTime); + case HoodieTimeline.DELTA_COMMIT_ACTION: + case HoodieTimeline.LOG_COMPACTION_ACTION: + return makeDeltaFileName(timestampWithCompletionTime); + case HoodieTimeline.RESTORE_ACTION: + return makeRestoreFileName(timestampWithCompletionTime); + case HoodieTimeline.REPLACE_COMMIT_ACTION: + return makeReplaceFileName(timestampWithCompletionTime); + case HoodieTimeline.INDEXING_ACTION: + return makeIndexCommitFileName(timestampWithCompletionTime); + case HoodieTimeline.SCHEMA_COMMIT_ACTION: + return makeSchemaFileName(timestampWithCompletionTime); + default: + throw new IllegalArgumentException("Cannot get complete instant's file name for unknown action " + + instant.getAction()); + } + } + + @Override + public String getFileName(HoodieInstant instant) { + if (instant.isCompleted()) { + return getCompleteFileName(instant, instant.getCompletionTime()); + } + + return getPendingFileName(instant); + } + + @Override + public String getFileName(String completionTime, HoodieInstant instant) { + ValidationUtils.checkState(instant.isCompleted()); + return getCompleteFileName(instant, completionTime); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameParserV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameParserV2.java new file mode 100644 index 000000000000..58df6b8b4291 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantFileNameParserV2.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; + +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class InstantFileNameParserV2 implements InstantFileNameParser { + // Instant like 20230104152218702.commit.request, 20230104152218702.inflight and 20230104152218702_20230104152630238.commit + private static final Pattern NAME_FORMAT = + Pattern.compile("^(\\d+(_\\d+)?)(\\.\\w+)(\\.\\D+)?$"); + + public String extractTimestamp(String fileName) throws IllegalArgumentException { + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + String timestamp = matcher.group(1); + return timestamp.contains(HoodieInstant.UNDERSCORE) ? timestamp.split(HoodieInstant.UNDERSCORE)[0] : timestamp; + } + + throw new IllegalArgumentException("Failed to retrieve timestamp from name: " + + String.format(HoodieInstant.FILE_NAME_FORMAT_ERROR, fileName)); + } + + public String getTimelineFileExtension(String fileName) { + Objects.requireNonNull(fileName); + + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + return fileName.substring(matcher.group(1).length()); + } + return HoodieInstant.EMPTY_FILE_EXTENSION; + } + +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantGeneratorV2.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantGeneratorV2.java new file mode 100644 index 000000000000..6b72afe0f604 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/InstantGeneratorV2.java @@ -0,0 +1,197 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.storage.StoragePathInfo; + +import java.util.Date; +import java.util.Objects; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +public class InstantGeneratorV2 implements InstantGenerator { + // Instant like 20230104152218702.commit.request, 20230104152218702.inflight and 20230104152218702_20230104152630238.commit + private static final Pattern NAME_FORMAT = + Pattern.compile("^(\\d+(_\\d+)?)(\\.\\w+)(\\.\\D+)?$"); + + private static final String DELIMITER = "."; + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp) { + return new HoodieInstant(state, action, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime) { + return new HoodieInstant(state, action, timestamp, completionTime, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(HoodieInstant.State state, String action, String timestamp, String completionTime, boolean isLegacy) { + return new HoodieInstant(state, action, timestamp, completionTime, isLegacy, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant createNewInstant(StoragePathInfo pathInfo) { + // First read the instant timestamp. [==>20170101193025<==].commit + String fileName = pathInfo.getPath().getName(); + String timestamp = null; + String completionTime = null; + String action = null; + HoodieInstant.State state = HoodieInstant.State.NIL; + boolean isLegacy = false; + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + String[] timestamps = matcher.group(1).split(HoodieInstant.UNDERSCORE); + timestamp = timestamps[0]; + if (matcher.group(3).equals(HoodieTimeline.INFLIGHT_EXTENSION)) { + // This is to support backwards compatibility on how in-flight commit files were written + // General rule is inflight extension is ..inflight, but for commit it is .inflight + action = HoodieTimeline.COMMIT_ACTION; + state = HoodieInstant.State.INFLIGHT; + } else { + action = matcher.group(3).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING); + if (matcher.groupCount() == 4 && matcher.group(4) != null) { + state = HoodieInstant.State.valueOf(matcher.group(4).replaceFirst(DELIMITER, StringUtils.EMPTY_STRING).toUpperCase()); + } else { + // Like 20230104152218702.commit + state = HoodieInstant.State.COMPLETED; + } + } + if (state == HoodieInstant.State.COMPLETED) { + if (timestamps.length > 1) { + completionTime = timestamps[1]; + } else { + // for backward compatibility with 0.x release. + completionTime = HoodieInstantTimeGenerator.formatDate(new Date(pathInfo.getModificationTime())); + isLegacy = true; + } + } else { + completionTime = null; + } + } else { + throw new IllegalArgumentException("Failed to construct HoodieInstant: " + + String.format(HoodieInstant.FILE_NAME_FORMAT_ERROR, fileName)); + } + return new HoodieInstant(state, action, timestamp, completionTime, isLegacy, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + public String extractTimestamp(String fileName) throws IllegalArgumentException { + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + String timestamp = matcher.group(1); + return timestamp.contains(HoodieInstant.UNDERSCORE) ? timestamp.split(HoodieInstant.UNDERSCORE)[0] : timestamp; + } + + throw new IllegalArgumentException("Failed to retrieve timestamp from name: " + + String.format(HoodieInstant.FILE_NAME_FORMAT_ERROR, fileName)); + } + + public String getTimelineFileExtension(String fileName) { + Objects.requireNonNull(fileName); + + Matcher matcher = NAME_FORMAT.matcher(fileName); + if (matcher.find()) { + return fileName.substring(matcher.group(1).length()); + } + return HoodieInstant.EMPTY_FILE_EXTENSION; + } + + @Override + public HoodieInstant getRequestedInstant(final HoodieInstant instant) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, instant.getAction(), instant.requestedTime(), InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCleanRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCleanInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + // Returns Log compaction requested instant + @Override + public HoodieInstant getLogCompactionRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.LOG_COMPACTION_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + // Returns Log compaction inflight instant + @Override + public HoodieInstant getLogCompactionInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getReplaceCommitRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getReplaceCommitInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getClusteringCommitRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getClusteringCommitInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getRollbackRequestedInstant(HoodieInstant instant) { + return instant.isRequested() ? instant : getRequestedInstant(instant); + } + + @Override + public HoodieInstant getRestoreRequestedInstant(HoodieInstant instant) { + return instant.isRequested() ? instant : getRequestedInstant(instant); + } + + @Override + public HoodieInstant getIndexRequestedInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.INDEXING_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } + + @Override + public HoodieInstant getIndexInflightInstant(final String timestamp) { + return new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.INDEXING_ACTION, timestamp, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/TimelineV2Factory.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/TimelineV2Factory.java new file mode 100644 index 000000000000..54d66cb7bb1b --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/versioning/v2/TimelineV2Factory.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.table.timeline.versioning.v2; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.ArchivedTimelineLoader; +import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.util.Option; + +import java.util.function.Function; +import java.util.stream.Stream; + +public class TimelineV2Factory extends TimelineFactory { + + private final TimelineLayout layout; + + public TimelineV2Factory(TimelineLayout layout) { + this.layout = layout; + } + + @Override + public HoodieTimeline createDefaultTimeline(Stream instants, Function> details) { + return new BaseTimelineV2(instants, details); + } + + @Override + public HoodieActiveTimeline createActiveTimeline() { + return new ActiveTimelineV2(); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient) { + return new ArchivedTimelineV2(metaClient); + } + + @Override + public HoodieArchivedTimeline createArchivedTimeline(HoodieTableMetaClient metaClient, String startTs) { + return new ArchivedTimelineV2(metaClient, startTs); + } + + @Override + public ArchivedTimelineLoader createArchivedTimelineLoader() { + return new ArchivedTimelineLoaderV2(); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient) { + return new ActiveTimelineV2(metaClient); + } + + @Override + public HoodieActiveTimeline createActiveTimeline(HoodieTableMetaClient metaClient, boolean applyLayoutFilter) { + return new ActiveTimelineV2(metaClient, applyLayoutFilter); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient) { + return new CompletionTimeQueryViewV2(metaClient); + } + + @Override + public CompletionTimeQueryView createCompletionTimeQueryView(HoodieTableMetaClient metaClient, String eagerInstant) { + return new CompletionTimeQueryViewV2(metaClient, eagerInstant); + } +} 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 0c72359b1102..a8098676403e 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 @@ -70,10 +70,13 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; +import static org.apache.hudi.common.table.timeline.InstantComparison.EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * Common thread-safe implementation for multiple TableFileSystemView Implementations. @@ -114,7 +117,7 @@ public abstract class AbstractTableFileSystemView implements SyncableFileSystemV */ protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; - this.completionTimeQueryView = new CompletionTimeQueryView(metaClient); + this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); refreshTimeline(visibleActiveTimeline); resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline); this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); @@ -142,7 +145,7 @@ protected void refreshTimeline(HoodieTimeline visibleActiveTimeline) { * Refresh the completion time query view. */ protected void refreshCompletionTimeQueryView() { - this.completionTimeQueryView = new CompletionTimeQueryView(metaClient); + this.completionTimeQueryView = metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient); } /** @@ -275,7 +278,7 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { // Duplicate key error when insert_overwrite same partition in multi writer, keep the instant with greater timestamp when the file group id conflicts Map replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue, - (instance1, instance2) -> HoodieTimeline.compareTimestamps(instance1.getTimestamp(), HoodieTimeline.LESSER_THAN, instance2.getTimestamp()) ? instance2 : instance1)); + (instance1, instance2) -> compareTimestamps(instance1.requestedTime(), LESSER_THAN, instance2.requestedTime()) ? instance2 : instance1)); resetReplacedFileGroups(replacedFileGroups); LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + replacedTimeline.countInstants() + " instants, " + replacedFileGroups.size() + " replaced file groups"); @@ -749,7 +752,7 @@ private Stream getLatestBaseFilesBeforeOrOnFromCache(String part return fetchAllStoredFileGroups(partitionPath) .filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime)) .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles() - .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime + .filter(baseFile -> compareTimestamps(baseFile.getCommitTime(), LESSER_THAN_OR_EQUALS, maxCommitTime )) .filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst())) .filter(Option::isPresent).map(Option::get) @@ -766,7 +769,7 @@ public final Option getBaseFileOn(String partitionStr, String in return Option.empty(); } else { return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() - .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, + .filter(baseFile -> compareTimestamps(baseFile.getCommitTime(), EQUALS, instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(partitionPath, df) && !isBaseFileDueToPendingClustering(df)).findFirst().orElse(null)) .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); } @@ -1586,7 +1589,7 @@ private boolean isFileGroupReplacedBefore(HoodieFileGroupId fileGroupId, String return false; } - return HoodieTimeline.compareTimestamps(instant, GREATER_THAN, hoodieInstantOption.get().getTimestamp()); + return compareTimestamps(instant, GREATER_THAN, hoodieInstantOption.get().requestedTime()); } private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, String instant) { @@ -1595,7 +1598,7 @@ private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, Str return false; } - return HoodieTimeline.compareTimestamps(instant, GREATER_THAN_OR_EQUALS, hoodieInstantOption.get().getTimestamp()); + return compareTimestamps(instant, GREATER_THAN_OR_EQUALS, hoodieInstantOption.get().requestedTime()); } private boolean isFileGroupReplacedAfterOrOn(HoodieFileGroupId fileGroupId, String instant) { @@ -1604,7 +1607,7 @@ private boolean isFileGroupReplacedAfterOrOn(HoodieFileGroupId fileGroupId, Stri return false; } - return HoodieTimeline.compareTimestamps(instant, LESSER_THAN_OR_EQUALS, hoodieInstantOption.get().getTimestamp()); + return compareTimestamps(instant, LESSER_THAN_OR_EQUALS, hoodieInstantOption.get().requestedTime()); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index d132b19c3ded..24ebadb45463 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -411,7 +411,7 @@ protected void addReplacedFileGroups(final Map @Override protected void removeReplacedFileIdsAtInstants(Set instants) { - fgIdToReplaceInstants.entrySet().removeIf(entry -> instants.contains(entry.getValue().getTimestamp())); + fgIdToReplaceInstants.entrySet().removeIf(entry -> instants.contains(entry.getValue().requestedTime())); } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index 36ab0edf4fbb..078dc38ae162 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -94,7 +94,7 @@ protected void maySyncIncrementally() { HoodieTimeline newTimeline = metaClient.reloadActiveTimeline().filterCompletedOrMajorOrMinorCompactionInstants(); try { if (incrementalTimelineSyncEnabled) { - TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(oldTimeline, newTimeline); + TimelineDiffResult diffResult = TimelineDiffHelper.getNewInstantsForIncrementalSync(metaClient, oldTimeline, newTimeline); if (diffResult.canSyncIncrementally()) { LOG.info("Doing incremental sync"); // need to refresh the completion time query view @@ -179,7 +179,7 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff */ private void removePendingCompactionInstant(HoodieInstant instant) throws IOException { LOG.info("Removing completed compaction instant (" + instant + ")"); - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp()); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, instant.requestedTime()); removePendingCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan) .map(instantPair -> Pair.of(instantPair.getValue().getKey(), CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue())))); @@ -193,7 +193,7 @@ private void removePendingCompactionInstant(HoodieInstant instant) throws IOExce */ private void removePendingLogCompactionInstant(HoodieInstant instant) throws IOException { LOG.info("Removing completed log compaction instant (" + instant + ")"); - HoodieCompactionPlan plan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp()); + HoodieCompactionPlan plan = CompactionUtils.getLogCompactionPlan(metaClient, instant.requestedTime()); removePendingLogCompactionOperations(CompactionUtils.getPendingCompactionOperations(instant, plan) .map(instantPair -> Pair.of(instantPair.getValue().getKey(), CompactionOperation.convertFromAvroRecordInstance(instantPair.getValue().getValue())))); @@ -207,7 +207,7 @@ private void removePendingLogCompactionInstant(HoodieInstant instant) throws IOE */ private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing pending compaction instant (" + instant + ")"); - HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp()); + HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan(metaClient, instant.requestedTime()); List> pendingOps = CompactionUtils.getPendingCompactionOperations(instant, compactionPlan) .map(p -> Pair.of(p.getValue().getKey(), @@ -237,7 +237,7 @@ private void addPendingCompactionInstant(HoodieTimeline timeline, HoodieInstant */ private void addPendingLogCompactionInstant(HoodieInstant instant) throws IOException { LOG.info("Syncing pending log compaction instant (" + instant + ")"); - HoodieCompactionPlan compactionPlan = CompactionUtils.getLogCompactionPlan(metaClient, instant.getTimestamp()); + HoodieCompactionPlan compactionPlan = CompactionUtils.getLogCompactionPlan(metaClient, instant.requestedTime()); List> pendingOps = CompactionUtils.getPendingCompactionOperations(instant, compactionPlan) .map(p -> Pair.of(p.getValue().getKey(), @@ -257,7 +257,7 @@ private void addPendingLogCompactionInstant(HoodieInstant instant) throws IOExce private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing committed instant (" + instant + ")"); HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); updatePartitionWriteFileGroups(commitMetadata.getPartitionToWriteStats(), timeline, instant); LOG.info("Done Syncing committed instant (" + instant + ")"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index 755b13ba79e1..7ebea7b4f799 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.dto.BaseFileDTO; import org.apache.hudi.common.table.timeline.dto.ClusteringOpDTO; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; @@ -181,7 +182,7 @@ private T executeRequest(String requestPath, Map queryParame queryParameters.forEach(builder::addParameter); // Adding mandatory parameters - Last instants affecting file-slice - timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.getTimestamp())); + timeline.lastInstant().ifPresent(instant -> builder.addParameter(LAST_INSTANT_TS, instant.requestedTime())); builder.addParameter(TIMELINE_HASH, timeline.getTimelineHash()); String url = builder.toString(); @@ -500,7 +501,8 @@ public Stream> getFileGroupsInPendingClus try { List dtos = executeRequest(PENDING_CLUSTERING_FILEGROUPS_URL, paramsMap, CLUSTERING_OP_DTOS_REFERENCE, RequestMethod.GET); - return dtos.stream().map(ClusteringOpDTO::toClusteringOperation); + InstantGenerator factory = metaClient.getInstantGenerator(); + return dtos.stream().map(dto -> ClusteringOpDTO.toClusteringOperation(dto, factory)); } catch (IOException e) { throw new HoodieRemoteException(e); } @@ -511,7 +513,8 @@ public Option getLastInstant() { Map paramsMap = getParams(); try { List instants = executeRequest(LAST_INSTANT_URL, paramsMap, INSTANT_DTOS_REFERENCE, RequestMethod.GET); - return Option.fromJavaOptional(instants.stream().map(InstantDTO::toInstant).findFirst()); + return Option.fromJavaOptional(instants.stream() + .map(dto -> InstantDTO.toInstant(dto, metaClient.getInstantGenerator())).findFirst()); } catch (IOException e) { throw new HoodieRemoteException(e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index a1ef5abd09ae..adee4ffcdc90 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -48,6 +48,10 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * A file-system view implementation on top of embedded Rocks DB store. For each table : 3 column Family is added for * storing (1) File-Slices and Data Files for View lookups (2) Pending compaction operations (3) Partitions tracked @@ -466,7 +470,7 @@ public Option fetchLatestFileSlice(String partitionPath, String fileI .map(Pair::getValue).reduce(null, (x, y) -> ((x == null) ? y : (y == null) ? null - : HoodieTimeline.compareTimestamps(x.getBaseInstantTime(), HoodieTimeline.GREATER_THAN, y.getBaseInstantTime() + : compareTimestamps(x.getBaseInstantTime(), GREATER_THAN, y.getBaseInstantTime() ) ? x : y))); } @@ -480,10 +484,10 @@ public Option fetchLatestFileSliceBeforeOrOn(String partitionPath, St .prefixSearch(schemaHelper.getColFamilyForView(), schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId)) .map(Pair::getValue) - .filter(fileSlice -> fileSlice != null && HoodieTimeline.compareTimestamps(fileSlice.getBaseInstantTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, completionTime.get())) + .filter(fileSlice -> fileSlice != null && compareTimestamps(fileSlice.getBaseInstantTime(), LESSER_THAN_OR_EQUALS, completionTime.get())) .reduce(null, (x, y) -> x == null ? y - : HoodieTimeline.compareTimestamps(x.getBaseInstantTime(), HoodieTimeline.GREATER_THAN, y.getBaseInstantTime() + : compareTimestamps(x.getBaseInstantTime(), GREATER_THAN, y.getBaseInstantTime() ) ? x : y)); } @@ -497,7 +501,7 @@ protected Option fetchLatestBaseFile(String partitionPath, Strin .map(Pair::getValue).reduce(null, (x, y) -> ((x == null) ? y : (y == null) ? null - : HoodieTimeline.compareTimestamps(x.getCommitTime(), HoodieTimeline.GREATER_THAN, y.getCommitTime()) + : compareTimestamps(x.getCommitTime(), GREATER_THAN, y.getCommitTime()) ? x : y))); } @@ -545,7 +549,7 @@ protected void addReplacedFileGroups(final Map protected void removeReplacedFileIdsAtInstants(Set instants) { //TODO can we make this more efficient by storing reverse mapping (Instant -> FileGroupId) as well? Stream keysToDelete = rocksDB.prefixSearch(schemaHelper.getColFamilyForReplacedFileGroups(), "") - .filter(entry -> instants.contains(entry.getValue().getTimestamp())) + .filter(entry -> instants.contains(entry.getValue().requestedTime())) .map(Pair::getKey); rocksDB.writeBatch(batch -> diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index 30b545456d3f..1f4eb3c1525f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -210,7 +210,7 @@ public Stream fetchAllStoredFileGroups() { protected void removeReplacedFileIdsAtInstants(Set instants) { //TODO should we make this more efficient by having reverse mapping of instant to file group id? Stream fileIdsToRemove = fgIdToReplaceInstants.entrySet().stream().map(entry -> { - if (instants.contains(entry.getValue().getTimestamp())) { + if (instants.contains(entry.getValue().requestedTime())) { return Option.of(entry.getKey()); } else { return Option.ofNullable((HoodieFileGroupId) null); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java index f1488f313184..4ac802e2314c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CleanerUtils.java @@ -28,10 +28,10 @@ import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; import org.apache.hudi.common.model.HoodieTimelineTimeZone; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataMigrator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataV1MigrationHandler; import org.apache.hudi.common.table.timeline.versioning.clean.CleanMetadataV2MigrationHandler; @@ -50,6 +50,8 @@ import java.util.stream.Collectors; import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * Utils for clean action. @@ -140,7 +142,7 @@ public static Option getEarliestCommitToRetain( if (nthInstant.compareTo(earliestPendingCommits.get()) <= 0) { return Option.of(nthInstant); } else { - return completedCommitsTimeline.findInstantsBefore(earliestPendingCommits.get().getTimestamp()).lastInstant(); + return completedCommitsTimeline.findInstantsBefore(earliestPendingCommits.get().requestedTime()).lastInstant(); } }).orElse(Option.empty()); } else { @@ -149,9 +151,9 @@ public static Option getEarliestCommitToRetain( } } else if (cleaningPolicy == HoodieCleaningPolicy.KEEP_LATEST_BY_HOURS) { ZonedDateTime latestDateTime = ZonedDateTime.ofInstant(latestInstant, timeZone.getZoneId()); - String earliestTimeToRetain = HoodieActiveTimeline.formatDate(Date.from(latestDateTime.minusHours(hoursRetained).toInstant())); - earliestCommitToRetain = Option.fromJavaOptional(completedCommitsTimeline.getInstantsAsStream().filter(i -> HoodieTimeline.compareTimestamps(i.getTimestamp(), - HoodieTimeline.GREATER_THAN_OR_EQUALS, earliestTimeToRetain)).findFirst()); + String earliestTimeToRetain = TimelineUtils.formatDate(Date.from(latestDateTime.minusHours(hoursRetained).toInstant())); + earliestCommitToRetain = Option.fromJavaOptional(completedCommitsTimeline.getInstantsAsStream().filter(i -> compareTimestamps(i.requestedTime(), + GREATER_THAN_OR_EQUALS, earliestTimeToRetain)).findFirst()); } return earliestCommitToRetain; } 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 691695dff45a..4b2f19a2327b 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 @@ -36,6 +36,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.collection.Pair; @@ -55,6 +56,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * Helper class to generate clustering plan from metadata. */ @@ -82,13 +86,13 @@ public static Stream> getAllPendingClu * clustering inflight commit. After HUDI-7905, all the requested and inflight clustering instants * use clustering action instead of replacecommit. */ - public static Option getInflightClusteringInstant(String timestamp, HoodieActiveTimeline activeTimeline) { + public static Option getInflightClusteringInstant(String timestamp, HoodieActiveTimeline activeTimeline, InstantGenerator factory) { HoodieTimeline pendingReplaceOrClusterTimeline = activeTimeline.filterPendingReplaceOrClusteringTimeline(); - HoodieInstant inflightInstant = HoodieTimeline.getClusteringCommitInflightInstant(timestamp); + HoodieInstant inflightInstant = factory.getClusteringCommitInflightInstant(timestamp); if (pendingReplaceOrClusterTimeline.containsInstant(inflightInstant)) { return Option.of(inflightInstant); } - inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(timestamp); + inflightInstant = factory.getReplaceCommitInflightInstant(timestamp); return Option.ofNullable(pendingReplaceOrClusterTimeline.containsInstant(inflightInstant) ? inflightInstant : null); } @@ -97,13 +101,13 @@ public static Option getInflightClusteringInstant(String timestam * clustering requested commit. After HUDI-7905, all the requested and inflight clustering instants * use clustering action instead of replacecommit. */ - public static Option getRequestedClusteringInstant(String timestamp, HoodieActiveTimeline activeTimeline) { + public static Option getRequestedClusteringInstant(String timestamp, HoodieActiveTimeline activeTimeline, InstantGenerator factory) { HoodieTimeline pendingReplaceOrClusterTimeline = activeTimeline.filterPendingReplaceOrClusteringTimeline(); - HoodieInstant requestedInstant = HoodieTimeline.getClusteringCommitRequestedInstant(timestamp); + HoodieInstant requestedInstant = factory.getClusteringCommitRequestedInstant(timestamp); if (pendingReplaceOrClusterTimeline.containsInstant(requestedInstant)) { return Option.of(requestedInstant); } - requestedInstant = HoodieTimeline.getReplaceCommitRequestedInstant(timestamp); + requestedInstant = factory.getReplaceCommitRequestedInstant(timestamp); return Option.ofNullable(pendingReplaceOrClusterTimeline.containsInstant(requestedInstant) ? requestedInstant : null); } @@ -152,9 +156,9 @@ public static boolean isClusteringOrReplaceCommitAction(String actionType) { * @param replaceInstant the instant of replacecommit action to check. * @return whether the instant is a clustering operation. */ - public static boolean isClusteringInstant(HoodieTimeline timeline, HoodieInstant replaceInstant) { + public static boolean isClusteringInstant(HoodieTimeline timeline, HoodieInstant replaceInstant, InstantGenerator factory) { return replaceInstant.getAction().equals(HoodieTimeline.CLUSTERING_ACTION) - || (replaceInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && getClusteringPlan(timeline, replaceInstant).isPresent()); + || (replaceInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION) && getClusteringPlan(timeline, replaceInstant, factory).isPresent()); } /** @@ -165,17 +169,18 @@ public static boolean isClusteringInstant(HoodieTimeline timeline, HoodieInstant * @return option of the replace metadata if present, else empty * @throws IOException */ - private static Option getRequestedReplaceMetadata(HoodieTimeline timeline, HoodieInstant pendingReplaceOrClusterInstant) throws IOException { + private static Option getRequestedReplaceMetadata(HoodieTimeline timeline, HoodieInstant pendingReplaceOrClusterInstant, + InstantGenerator factory) throws IOException { HoodieInstant requestedInstant; if (pendingReplaceOrClusterInstant.isInflight()) { // inflight replacecommit files don't have clustering plan. // This is because replacecommit inflight can have workload profile for 'insert_overwrite'. // Get the plan from corresponding requested instant. - requestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, pendingReplaceOrClusterInstant.getAction(), pendingReplaceOrClusterInstant.getTimestamp()); + requestedInstant = factory.createNewInstant(HoodieInstant.State.REQUESTED, pendingReplaceOrClusterInstant.getAction(), pendingReplaceOrClusterInstant.requestedTime()); } else if (pendingReplaceOrClusterInstant.isRequested()) { requestedInstant = pendingReplaceOrClusterInstant; } else { - requestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, pendingReplaceOrClusterInstant.getTimestamp()); + requestedInstant = factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, pendingReplaceOrClusterInstant.requestedTime()); } Option content = Option.empty(); try { @@ -184,7 +189,7 @@ private static Option getRequestedReplaceMetadat if (e.getCause() instanceof FileNotFoundException && pendingReplaceOrClusterInstant.isCompleted()) { // For clustering instants, completed instant is also a replace commit instant. For input replace commit instant, // it is not known whether requested instant is CLUSTER or REPLACE_COMMIT_ACTION. So we need to query both. - requestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, pendingReplaceOrClusterInstant.getTimestamp()); + requestedInstant = factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, pendingReplaceOrClusterInstant.requestedTime()); content = timeline.getInstantDetails(requestedInstant); } } @@ -202,7 +207,7 @@ private static Option getRequestedReplaceMetadat * @return option of the replace metadata if present, else empty */ public static Option> getClusteringPlan(HoodieTableMetaClient metaClient, HoodieInstant pendingReplaceInstant) { - return getClusteringPlan(metaClient.getActiveTimeline(), pendingReplaceInstant); + return getClusteringPlan(metaClient.getActiveTimeline(), pendingReplaceInstant, metaClient.getInstantGenerator()); } /** @@ -211,15 +216,15 @@ public static Option> getClusteringPla * @param pendingReplaceInstant * @return */ - public static Option> getClusteringPlan(HoodieTimeline timeline, HoodieInstant pendingReplaceInstant) { + public static Option> getClusteringPlan(HoodieTimeline timeline, HoodieInstant pendingReplaceInstant, InstantGenerator factory) { try { - Option requestedReplaceMetadata = getRequestedReplaceMetadata(timeline, pendingReplaceInstant); + Option requestedReplaceMetadata = getRequestedReplaceMetadata(timeline, pendingReplaceInstant, factory); if (requestedReplaceMetadata.isPresent() && WriteOperationType.CLUSTER.name().equals(requestedReplaceMetadata.get().getOperationType())) { return Option.of(Pair.of(pendingReplaceInstant, requestedReplaceMetadata.get().getClusteringPlan())); } return Option.empty(); } catch (IOException e) { - throw new HoodieIOException("Error reading clustering plan " + pendingReplaceInstant.getTimestamp(), e); + throw new HoodieIOException("Error reading clustering plan " + pendingReplaceInstant.requestedTime(), e); } } @@ -326,8 +331,9 @@ private static Map buildMetrics(List fileSlices) { } public static List getPendingClusteringInstantTimes(HoodieTableMetaClient metaClient) { + InstantGenerator factory = metaClient.getInstantGenerator(); return metaClient.getActiveTimeline().filterPendingReplaceOrClusteringTimeline().getInstantsAsStream() - .filter(instant -> isClusteringInstant(metaClient.getActiveTimeline(), instant)) + .filter(instant -> isClusteringInstant(metaClient.getActiveTimeline(), instant, factory)) .collect(Collectors.toList()); } @@ -342,6 +348,7 @@ public static List getPendingClusteringInstantTimes(HoodieTableMe */ public static Option getEarliestInstantToRetainForClustering( HoodieActiveTimeline activeTimeline, HoodieTableMetaClient metaClient, HoodieCleaningPolicy cleanerPolicy) throws IOException { + InstantGenerator factory = metaClient.getInstantGenerator(); Option oldestInstantToRetain = Option.empty(); HoodieTimeline replaceOrClusterTimeline = activeTimeline.getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.CLUSTERING_ACTION)); if (!replaceOrClusterTimeline.empty()) { @@ -351,7 +358,7 @@ public static Option getEarliestInstantToRetainForClustering( // The first clustering instant of which timestamp is greater than or equal to the earliest commit to retain of // the clean metadata. HoodieInstant cleanInstant = cleanInstantOpt.get(); - HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(metaClient, cleanInstant.isRequested() ? cleanInstant : HoodieTimeline.getCleanRequestedInstant(cleanInstant.getTimestamp())); + HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(metaClient, cleanInstant.isRequested() ? cleanInstant : factory.getCleanRequestedInstant(cleanInstant.requestedTime())); Option earliestInstantToRetain = Option.ofNullable(cleanerPlan.getEarliestInstantToRetain()).map(HoodieActionInstant::getTimestamp); String retainLowerBound; Option earliestReplacedSavepointInClean = getEarliestReplacedSavepointInClean(activeTimeline, cleanerPolicy, cleanerPlan); @@ -367,7 +374,7 @@ public static Option getEarliestInstantToRetainForClustering( // the cleaner would have removed all the file groups until then. But there is a catch to this logic, // while cleaner is running if there is a pending replacecommit then those files are not cleaned. // TODO: This case has to be handled. HUDI-6352 - retainLowerBound = cleanInstant.getTimestamp(); + retainLowerBound = cleanInstant.requestedTime(); } oldestInstantToRetain = replaceOrClusterTimeline.findInstantsAfterOrEquals(retainLowerBound).firstInstant(); } else { @@ -403,7 +410,7 @@ public static Option getEarliestReplacedSavepointInClean(HoodieActiveTim // When earliestToRetainTs is greater than first savepoint timestamp and there are no // replace commits between the first savepoint and the earliestToRetainTs, we can set the // earliestSavepointOpt to empty as there was no cleaning blocked due to savepoint - if (HoodieTimeline.compareTimestamps(earliestInstantToRetain, HoodieTimeline.GREATER_THAN, earliestSavepoint)) { + if (compareTimestamps(earliestInstantToRetain, GREATER_THAN, earliestSavepoint)) { HoodieTimeline replaceTimeline = activeTimeline.getCompletedReplaceTimeline().findInstantsInClosedRange(earliestSavepoint, earliestInstantToRetain); if (!replaceTimeline.empty()) { return Option.of(earliestSavepoint); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java index 30398af1f0fb..988a620c6d28 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java @@ -23,10 +23,10 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; @@ -125,9 +125,10 @@ private static HoodieCommitMetadata buildMetadataFromStats(List return commitMetadata; } - public static Option buildMetadataFromInstant(HoodieDefaultTimeline timeline, HoodieInstant instant) { + public static Option buildMetadataFromInstant(HoodieTimeline timeline, HoodieInstant instant) { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); @@ -180,11 +181,12 @@ public static Set> flattenPartitionToReplaceFileIds(Map getValidCheckpointForCurrentWriter(HoodieTimeline timeline, String checkpointKey, String keyToLookup) { + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); return (Option) timeline.getWriteTimeline().filterCompletedInstants().getReverseOrderedInstants() .map(instant -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() + .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); // process commits only with checkpoint entries String checkpointValue = commitMetadata.getMetadata(checkpointKey); if (StringUtils.nonEmpty(checkpointValue)) { 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 5c2270db919c..4a77fcbf9d51 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 @@ -129,7 +129,7 @@ public static List> getAllPendingCompa Function getFilteredTimelineByActionType = (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingCompactionTimeline(); // Hoodie requested instant supplier - Function requestedInstantSupplier = HoodieTimeline::getCompactionRequestedInstant; + Function requestedInstantSupplier = metaClient.getInstantGenerator()::getCompactionRequestedInstant; return getCompactionPlansByTimeline(metaClient, getFilteredTimelineByActionType, requestedInstantSupplier); } @@ -143,7 +143,7 @@ public static List> getAllPendingLogCo Function filteredTimelineSupplier = (hoodieTableMetaClient) -> hoodieTableMetaClient.getActiveTimeline().filterPendingLogCompactionTimeline(); // Hoodie requested instant supplier - Function requestedInstantSupplier = HoodieTimeline::getLogCompactionRequestedInstant; + Function requestedInstantSupplier = metaClient.getInstantGenerator()::getLogCompactionRequestedInstant; return getCompactionPlansByTimeline(metaClient, filteredTimelineSupplier, requestedInstantSupplier); } @@ -160,7 +160,7 @@ private static List> getCompactionPlan Function requestedInstantWrapper) { List filteredInstants = filteredTimelineSupplier.apply(metaClient).getInstants(); return filteredInstants.stream() - .map(instant -> Pair.of(instant, getCompactionPlan(metaClient, requestedInstantWrapper.apply(instant.getTimestamp())))) + .map(instant -> Pair.of(instant, getCompactionPlan(metaClient, requestedInstantWrapper.apply(instant.requestedTime())))) .collect(Collectors.toList()); } @@ -169,7 +169,7 @@ private static List> getCompactionPlan * because we use same HoodieCompactionPlan for both the operations. */ public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, String compactionInstant) { - HoodieInstant compactionRequestedInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstant); + HoodieInstant compactionRequestedInstant = metaClient.getInstantGenerator().getCompactionRequestedInstant(compactionInstant); return getCompactionPlan(metaClient, compactionRequestedInstant); } @@ -178,7 +178,7 @@ public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaC * because we use same HoodieCompactionPlan for both the operations. */ public static HoodieCompactionPlan getLogCompactionPlan(HoodieTableMetaClient metaClient, String logCompactionInstant) { - HoodieInstant logCompactionRequestedInstant = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstant); + HoodieInstant logCompactionRequestedInstant = metaClient.getInstantGenerator().getLogCompactionRequestedInstant(logCompactionInstant); return getCompactionPlan(metaClient, logCompactionRequestedInstant); } @@ -259,7 +259,7 @@ public static Stream ops = compactionPlan.getOperations(); if (null != ops) { return ops.stream().map(op -> Pair.of(new HoodieFileGroupId(op.getPartitionPath(), op.getFileId()), - Pair.of(instant.getTimestamp(), op))); + Pair.of(instant.requestedTime(), op))); } else { return Stream.empty(); } @@ -308,7 +308,7 @@ public static Option> getDeltaCommitsSinceLa latestInstant = lastCompaction.get(); // timeline containing the delta commits after the latest completed compaction commit, // and the completed compaction commit instant - return Option.of(Pair.of(deltaCommits.findInstantsModifiedAfterByCompletionTime(latestInstant.getTimestamp()), latestInstant)); + return Option.of(Pair.of(deltaCommits.findInstantsModifiedAfterByCompletionTime(latestInstant.requestedTime()), latestInstant)); } else { if (deltaCommits.countInstants() > 0) { latestInstant = deltaCommits.firstInstant().get(); @@ -337,13 +337,13 @@ public static Option> getDeltaCommitsSinceLa // timeline containing the delta commits after the latest completed compaction commit, // and the completed compaction commit instant return Option.of(Pair.of(deltaCommits.findInstantsAfter( - latestInstant.getTimestamp(), Integer.MAX_VALUE), lastCompaction.get())); + latestInstant.requestedTime(), Integer.MAX_VALUE), lastCompaction.get())); } else { if (deltaCommits.countInstants() > 0) { latestInstant = deltaCommits.firstInstant().get(); // timeline containing all the delta commits, and the first delta commit instant return Option.of(Pair.of(deltaCommits.findInstantsAfterOrEquals( - latestInstant.getTimestamp(), Integer.MAX_VALUE), latestInstant)); + latestInstant.requestedTime(), Integer.MAX_VALUE), latestInstant)); } else { return Option.empty(); } 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 0865b7cdada1..51f9587433c2 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 @@ -21,8 +21,12 @@ import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.internal.schema.InternalSchema; @@ -36,6 +40,7 @@ import com.github.benmanes.caffeine.cache.Cache; import com.github.benmanes.caffeine.cache.Caffeine; import org.apache.avro.Schema; +import org.apache.hudi.storage.StoragePathInfo; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -114,12 +119,12 @@ private static TreeMap getHistoricalSchemas(HoodieTableMet private static Option getSchemaByReadingCommitFile(long versionID, HoodieTableMetaClient metaClient) { try { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - List instants = timeline.getInstantsAsStream().filter(f -> f.getTimestamp().equals(String.valueOf(versionID))).collect(Collectors.toList()); + List instants = timeline.getInstantsAsStream().filter(f -> f.requestedTime().equals(String.valueOf(versionID))).collect(Collectors.toList()); if (instants.isEmpty()) { return Option.empty(); } byte[] data = timeline.getInstantDetails(instants.get(0)).get(); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instants.get(0), data, HoodieCommitMetadata.class); String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA); return SerDeHelper.fromJson(latestInternalSchemaStr); } catch (Exception e) { @@ -143,7 +148,7 @@ public static Pair, Option> getInternalSchemaAndAvroSchem byte[] data = timelineBeforeCurrentCompaction.getInstantDetails(lastInstantBeforeCurrentCompaction.get()).get(); HoodieCommitMetadata metadata; try { - metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + metadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstantBeforeCurrentCompaction.get(), data, HoodieCommitMetadata.class); } catch (Exception e) { throw new HoodieException(String.format("cannot read metadata from commit: %s", lastInstantBeforeCurrentCompaction.get()), e); } @@ -173,18 +178,22 @@ public static Pair, Option> getInternalSchemaAndAvroSchem * @param tablePath table path * @param storage {@link HoodieStorage} instance. * @param validCommits current validate commits, use to make up the commit file path/verify the validity of the history schema files + * @param fileNameParser InstantFileNameParser + * @param commitMetadataSerDe CommitMetadataSerDe + * @param instantGenerator InstantGenerator * @return a internalSchema. */ - public static InternalSchema getInternalSchemaByVersionId(long versionId, String tablePath, HoodieStorage storage, String validCommits) { + public static InternalSchema getInternalSchemaByVersionId(long versionId, String tablePath, HoodieStorage storage, String validCommits, + InstantFileNameParser fileNameParser, CommitMetadataSerDe commitMetadataSerDe, InstantGenerator instantGenerator) { String avroSchema = ""; Set commitSet = Arrays.stream(validCommits.split(",")).collect(Collectors.toSet()); List validateCommitList = - commitSet.stream().map(HoodieInstant::extractTimestamp).collect(Collectors.toList()); + commitSet.stream().map(fileNameParser::extractTimestamp).collect(Collectors.toList()); StoragePath hoodieMetaPath = new StoragePath(tablePath, HoodieTableMetaClient.METAFOLDER_NAME); //step1: StoragePath candidateCommitFile = commitSet.stream() - .filter(fileName -> HoodieInstant.extractTimestamp(fileName).equals(versionId + "")) + .filter(fileName -> fileNameParser.extractTimestamp(fileName).equals(versionId + "")) .findFirst().map(f -> new StoragePath(hoodieMetaPath, f)).orElse(null); if (candidateCommitFile != null) { try { @@ -194,7 +203,9 @@ public static InternalSchema getInternalSchemaByVersionId(long versionId, String } catch (IOException e) { throw e; } - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = commitMetadataSerDe.deserialize(instantGenerator.createNewInstant( + new StoragePathInfo(candidateCommitFile, -1, false, (short)0, 0L, 0L)), + data, HoodieCommitMetadata.class); String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA); avroSchema = metadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY); if (latestInternalSchemaStr != null) { @@ -224,9 +235,11 @@ public static InternalSchema getInternalSchemaByVersionId(long versionId, String } public static InternalSchema getInternalSchemaByVersionId(long versionId, HoodieTableMetaClient metaClient) { + InstantFileNameGenerator factory = metaClient.getInstantFileNameGenerator(); String validCommitLists = metaClient - .getCommitsAndCompactionTimeline().filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getFileName).collect(Collectors.joining(",")); - return getInternalSchemaByVersionId(versionId, metaClient.getBasePath().toString(), metaClient.getStorage(), validCommitLists); + .getCommitsAndCompactionTimeline().filterCompletedInstants().getInstantsAsStream().map(factory::getFileName).collect(Collectors.joining(",")); + return getInternalSchemaByVersionId(versionId, metaClient.getBasePath().toString(), metaClient.getStorage(), + validCommitLists, metaClient.getInstantFileNameParser(), metaClient.getCommitMetadataSerDe(), metaClient.getInstantGenerator()); } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java index aa775a36ed15..53fe63c9625b 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/MarkerUtils.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.marker.MarkerType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.storage.HoodieStorage; @@ -277,7 +278,8 @@ public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set currentInstants.removeAll(completedCommitInstants); Set missingFileIDs = currentInstants.stream().flatMap(instant -> { try { - return HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class) + TimelineLayout layout = TimelineLayout.fromVersion(activeTimeline.getTimelineLayoutVersion()); + return layout.getCommitMetadataSerDe().deserialize(instant, activeTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class) .getFileIdAndRelativePaths().keySet().stream(); } catch (Exception e) { return Stream.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 345bd3d35110..6ab23dff1173 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 @@ -87,11 +87,11 @@ private HoodieTableMetaClient getMetaClient() { public void persistHistorySchemaStr(String instantTime, String historySchemaStr) { cleanResidualFiles(); HoodieActiveTimeline timeline = getMetaClient().getActiveTimeline(); - HoodieInstant hoodieInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, SCHEMA_COMMIT_ACTION, instantTime); + HoodieInstant hoodieInstant = metaClient.createNewInstant(HoodieInstant.State.REQUESTED, SCHEMA_COMMIT_ACTION, instantTime); timeline.createNewInstant(hoodieInstant); byte[] writeContent = getUTF8Bytes(historySchemaStr); timeline.transitionRequestedToInflight(hoodieInstant, Option.empty()); - timeline.saveAsComplete(false, new HoodieInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.getTimestamp()), Option.of(writeContent)); + timeline.saveAsComplete(false, metaClient.createNewInstant(HoodieInstant.State.INFLIGHT, hoodieInstant.getAction(), hoodieInstant.requestedTime()), Option.of(writeContent)); LOG.info(String.format("persist history schema success on commit time: %s", instantTime)); } @@ -139,7 +139,7 @@ public void cleanOldFiles(List validateCommits) { private List getValidInstants() { return getMetaClient().getCommitsTimeline() - .filterCompletedInstants().getInstantsAsStream().map(f -> f.getTimestamp()).collect(Collectors.toList()); + .filterCompletedInstants().getInstantsAsStream().map(f -> f.requestedTime()).collect(Collectors.toList()); } @Override @@ -156,7 +156,7 @@ public String getHistorySchemaStrByGivenValidCommits(List validCommits) .filter(f -> f.isFile() && f.getPath().getName().endsWith(SCHEMA_COMMIT_ACTION)) .map(file -> file.getPath().getName()) .filter(Objects::nonNull) - .filter(f -> commitList.contains(HoodieInstant.extractTimestamp(f))).sorted().collect(Collectors.toList()); + .filter(f -> commitList.contains(getMetaClient().getInstantFileNameParser().extractTimestamp(f))).sorted().collect(Collectors.toList()); if (!validaSchemaFiles.isEmpty()) { StoragePath latestFilePath = new StoragePath(baseSchemaPath, validaSchemaFiles.get(validaSchemaFiles.size() - 1)); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java index 95827aefccd8..cef4dd9352f1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/BaseTableMetadata.java @@ -476,7 +476,7 @@ protected StorageConfiguration getStorageConf() { protected String getLatestDataInstantTime() { return dataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant() - .map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + .map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); } public boolean isMetadataTableInitialized() { 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 024737f3eba2..8fa2fafc7632 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 @@ -37,7 +37,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.HoodieTimer; import org.apache.hudi.common.util.Option; @@ -647,9 +647,9 @@ public Pair getLogRecordScanner(List latestMetadataInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - String latestMetadataInstantTime = latestMetadataInstant.map(HoodieInstant::getTimestamp).orElse(SOLO_COMMIT_TIMESTAMP); + String latestMetadataInstantTime = latestMetadataInstant.map(HoodieInstant::requestedTime).orElse(SOLO_COMMIT_TIMESTAMP); if (timeTravelInstant.isPresent()) { - latestMetadataInstantTime = HoodieTimeline.minTimestamp(latestMetadataInstantTime, timeTravelInstant.get()); + latestMetadataInstantTime = InstantComparison.minTimestamp(latestMetadataInstantTime, timeTravelInstant.get()); } boolean allowFullScan = allowFullScanOverride.orElseGet(() -> isFullScanAllowedForPartition(partitionName)); @@ -764,7 +764,7 @@ public Option getSyncedInstantTime() { if (metadataMetaClient != null) { Option latestInstant = metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant(); if (latestInstant.isPresent()) { - return Option.of(latestInstant.get().getTimestamp()); + return Option.of(latestInstant.get().requestedTime()); } } return Option.empty(); @@ -775,7 +775,7 @@ public Option getLatestCompactionTime() { if (metadataMetaClient != null) { Option latestCompaction = metadataMetaClient.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants().lastInstant(); if (latestCompaction.isPresent()) { - return Option.of(latestCompaction.get().getTimestamp()); + return Option.of(latestCompaction.get().requestedTime()); } } return Option.empty(); diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java index 523b8a3785b2..b3562b86d2b6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieMetadataPayload.java @@ -32,7 +32,7 @@ import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordGlobalLocation; import org.apache.hudi.common.model.HoodieRecordPayload; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.hash.ColumnIndexID; import org.apache.hudi.common.util.hash.FileIndexID; @@ -563,7 +563,7 @@ public static HoodieRecord createRecordIndexUpdate(String HoodieKey key = new HoodieKey(recordKey, MetadataPartitionType.RECORD_INDEX.getPartitionPath()); long instantTimeMillis = -1; try { - instantTimeMillis = HoodieActiveTimeline.parseDateFromInstantTime(instantTime).getTime(); + instantTimeMillis = TimelineUtils.parseDateFromInstantTime(instantTime).getTime(); } catch (Exception e) { throw new HoodieMetadataException("Failed to create metadata payload for record index. Instant time parsing for " + instantTime + " failed ", e); } diff --git a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java index 994ec6708bd4..4fd70d950a1c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java +++ b/hudi-common/src/main/java/org/apache/hudi/metadata/HoodieTableMetadataUtil.java @@ -68,9 +68,11 @@ import org.apache.hudi.common.table.log.HoodieMergedLogRecordScanner; import org.apache.hudi.common.table.log.HoodieUnMergedLogRecordScanner; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.TimelineFactory; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.CollectionUtils; @@ -145,7 +147,9 @@ import static org.apache.hudi.common.config.HoodieCommonConfig.SPILLABLE_DISK_MAP_TYPE; import static org.apache.hudi.common.config.HoodieReaderConfig.ENABLE_OPTIMIZED_LOG_BLOCKS_SCAN; import static org.apache.hudi.common.fs.FSUtils.getFileNameFromPath; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ConfigUtils.getReaderConfigs; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.common.util.StringUtils.isNullOrEmpty; @@ -757,8 +761,9 @@ public static HoodieData convertMetadataToColumnStatsRecords(Hoodi private static void reAddLogFilesFromRollbackPlan(HoodieTableMetaClient dataTableMetaClient, String instantTime, Map> partitionToFilesMap) { - HoodieInstant rollbackInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, instantTime); - HoodieInstant requested = HoodieTimeline.getRollbackRequestedInstant(rollbackInstant); + InstantGenerator factory = dataTableMetaClient.getInstantGenerator(); + HoodieInstant rollbackInstant = factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, instantTime); + HoodieInstant requested = factory.getRollbackRequestedInstant(rollbackInstant); try { HoodieRollbackPlan rollbackPlan = TimelineMetadataUtils.deserializeAvroMetadata( dataTableMetaClient.getActiveTimeline().readRollbackInfoAsBytes(requested).get(), HoodieRollbackPlan.class); @@ -1065,10 +1070,11 @@ public static HoodieTableFileSystemView getFileSystemView(HoodieTableMetaClient // default FileSystemView will not return any file slices even // though we may have initialized them. HoodieTimeline timeline = metaClient.getActiveTimeline(); + TimelineFactory factory = metaClient.getTimelineLayout().getTimelineFactory(); if (timeline.empty()) { - final HoodieInstant instant = new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, + final HoodieInstant instant = metaClient.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, metaClient.createNewInstantTime(false)); - timeline = new HoodieDefaultTimeline(Stream.of(instant), metaClient.getActiveTimeline()::getInstantDetails); + timeline = factory.createDefaultTimeline(Stream.of(instant), metaClient.getActiveTimeline()::getInstantDetails); } return new HoodieTableFileSystemView(metaClient, timeline); } @@ -1097,7 +1103,7 @@ private static List getPartitionFileSlices(HoodieTableMetaClient meta fileSliceStream = fsView.getLatestMergedFileSlicesBeforeOrOn( // including pending compaction instant as the last instant so that the finished delta commits // that start earlier than the compaction can be queried. - partition, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant().get().getTimestamp()); + partition, metaClient.getActiveTimeline().filterCompletedAndCompactionInstants().lastInstant().get().requestedTime()); } else { return Collections.emptyList(); } @@ -1289,7 +1295,7 @@ public static List> getLogFileColumnRangeM .withBasePath(datasetMetaClient.getBasePath()) .withLogFilePaths(Collections.singletonList(filePath)) .withBufferSize(maxBufferSize) - .withLatestInstantTime(datasetMetaClient.getActiveTimeline().getCommitsTimeline().lastInstant().get().getTimestamp()) + .withLatestInstantTime(datasetMetaClient.getActiveTimeline().getCommitsTimeline().lastInstant().get().requestedTime()) .withReaderSchema(writerSchemaOpt.get()) .withTableMetaClient(datasetMetaClient) .withLogRecordScannerCallback(records::add) @@ -1441,8 +1447,8 @@ public static Set getValidInstantTimestamps(HoodieTableMetaClient dataMe // 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 datasetPendingInstants = datasetTimeline.filterInflightsAndRequested().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); - Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + Set datasetPendingInstants = datasetTimeline.filterInflightsAndRequested().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); + Set validInstantTimestamps = datasetTimeline.filterCompletedInstants().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toSet()); // We should also add completed indexing delta commits in the metadata table, as they do not // have corresponding completed instant in the data table @@ -1450,7 +1456,7 @@ public static Set getValidInstantTimestamps(HoodieTableMetaClient dataMe metadataMetaClient.getActiveTimeline() .filter(instant -> instant.isCompleted() && isValidInstant(datasetPendingInstants, instant)) .getInstantsAsStream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .collect(Collectors.toList())); // For any rollbacks and restores, we cannot neglect the instants that they are rolling back. @@ -1458,17 +1464,17 @@ public static Set getValidInstantTimestamps(HoodieTableMetaClient dataMe // instant which we have a log block for. final String earliestInstantTime = validInstantTimestamps.isEmpty() ? SOLO_COMMIT_TIMESTAMP : Collections.min(validInstantTimestamps); datasetTimeline.getRollbackAndRestoreTimeline().filterCompletedInstants().getInstantsAsStream() - .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, earliestInstantTime)) - .forEach(instant -> validInstantTimestamps.addAll(getRollbackedCommits(instant, datasetTimeline))); + .filter(instant -> compareTimestamps(instant.requestedTime(), GREATER_THAN, earliestInstantTime)) + .forEach(instant -> validInstantTimestamps.addAll(getRollbackedCommits(instant, datasetTimeline, dataMetaClient.getInstantGenerator()))); // add restore and rollback instants from MDT. metadataMetaClient.getActiveTimeline().getRollbackAndRestoreTimeline().filterCompletedInstants() .filter(instant -> instant.getAction().equals(HoodieTimeline.RESTORE_ACTION) || instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) - .getInstants().forEach(instant -> validInstantTimestamps.add(instant.getTimestamp())); + .getInstants().forEach(instant -> validInstantTimestamps.add(instant.requestedTime())); metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants() - .filter(instant -> instant.getTimestamp().startsWith(SOLO_COMMIT_TIMESTAMP)) - .getInstants().forEach(instant -> validInstantTimestamps.add(instant.getTimestamp())); + .filter(instant -> instant.requestedTime().startsWith(SOLO_COMMIT_TIMESTAMP)) + .getInstants().forEach(instant -> validInstantTimestamps.add(instant.requestedTime())); return validInstantTimestamps; } @@ -1483,7 +1489,7 @@ private static boolean isValidInstant(Set datasetPendingInstants, Hoodie // only includes a deltacommit, // filter out any MDT instant that has pending corespondent dataset instant, // this comes from a case that one instant fails to commit after MDT had been committed. - return instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION) && !datasetPendingInstants.contains(instant.getTimestamp()); + return instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION) && !datasetPendingInstants.contains(instant.requestedTime()); } /** @@ -1513,7 +1519,7 @@ public static boolean isIndexingCommit( * @param instant The Rollback operation to read * @param timeline instant of timeline from dataset. */ - private static List getRollbackedCommits(HoodieInstant instant, HoodieActiveTimeline timeline) { + private static List getRollbackedCommits(HoodieInstant instant, HoodieActiveTimeline timeline, InstantGenerator factory) { try { List commitsToRollback; if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { @@ -1524,8 +1530,8 @@ private static List getRollbackedCommits(HoodieInstant instant, HoodieAc } catch (IOException e) { // if file is empty, fetch the commits to rollback from rollback.requested file HoodieRollbackPlan rollbackPlan = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.readRollbackInfoAsBytes(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, - instant.getTimestamp())).get(), HoodieRollbackPlan.class); + timeline.readRollbackInfoAsBytes(factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, + instant.requestedTime())).get(), HoodieRollbackPlan.class); commitsToRollback = Collections.singletonList(rollbackPlan.getInstantToRollback().getCommitTime()); LOG.warn("Had to fetch rollback info from requested instant since completed file is empty {}", instant); } @@ -1815,7 +1821,7 @@ public static HoodieRecordGlobalLocation getLocationFromRecordIndexInfo( } final java.util.Date instantDate = new java.util.Date(instantTime); - return new HoodieRecordGlobalLocation(partition, HoodieActiveTimeline.formatDate(instantDate), fileId); + return new HoodieRecordGlobalLocation(partition, HoodieInstantTimeGenerator.formatDate(instantDate), fileId); } /** @@ -1880,7 +1886,7 @@ public static HoodieData readRecordKeysFromFileSlices(HoodieEngine .withBasePath(basePath) .withLogFilePaths(logFilePaths) .withReaderSchema(HoodieAvroUtils.getRecordKeySchema()) - .withLatestInstantTime(metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::getTimestamp).orElse("")) + .withLatestInstantTime(metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::requestedTime).orElse("")) .withReverseReader(false) .withMaxMemorySizeInBytes(storageConf.getLong( MAX_MEMORY_FOR_COMPACTION.key(), DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES)) @@ -1960,7 +1966,7 @@ public static HoodieData readSecondaryKeysFromBaseFiles(HoodieEngi readerSchema = tableSchema; } return createSecondaryIndexGenerator(metaClient, engineType, logFilePaths, readerSchema, partition, dataFilePath, indexDefinition, - metaClient.getActiveTimeline().getCommitsTimeline().lastInstant().map(HoodieInstant::getTimestamp).orElse("")); + metaClient.getActiveTimeline().getCommitsTimeline().lastInstant().map(HoodieInstant::requestedTime).orElse("")); }); } @@ -1996,7 +2002,7 @@ public static HoodieData readSecondaryKeysFromFileSlices(HoodieEng readerSchema = tableSchema; } return createSecondaryIndexGenerator(metaClient, engineType, logFilePaths, readerSchema, partition, dataFilePath, indexDefinition, - metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::getTimestamp).orElse("")); + metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().map(HoodieInstant::requestedTime).orElse("")); }); } @@ -2487,7 +2493,7 @@ public DirectoryInfo(String relativePath, List pathInfos, Strin // Regular HUDI data file (base file or log file) String dataFileCommitTime = FSUtils.getCommitTime(pathInfo.getPath().getName()); // Limit the file listings to files which were created by successful commits before the maxInstant time. - if (!pendingDataInstants.contains(dataFileCommitTime) && HoodieTimeline.compareTimestamps(dataFileCommitTime, LESSER_THAN_OR_EQUALS, maxInstantTime)) { + if (!pendingDataInstants.contains(dataFileCommitTime) && compareTimestamps(dataFileCommitTime, LESSER_THAN_OR_EQUALS, maxInstantTime)) { filenameToSizeMap.put(pathInfo.getPath().getName(), pathInfo.getLength()); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java index a425a55db68f..6ced2d675a51 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieCommitMetadata.java @@ -18,7 +18,10 @@ package org.apache.hudi.common.model; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.versioning.v1.CommitMetadataSerDeV1; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FileIOUtils; @@ -36,6 +39,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -140,6 +145,46 @@ public void testGetFileSliceForFileGroupFromDeltaCommit() throws IOException { assertEquals("5.log", result.get().getValue().get(0)); } + @Test + public void testCommitMetadataSerde() throws Exception { + org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadata = new org.apache.hudi.avro.model.HoodieCommitMetadata(); + org.apache.hudi.avro.model.HoodieWriteStat writeStat1 = createWriteStat("111", "111base", Arrays.asList("1.log", "2.log")); + org.apache.hudi.avro.model.HoodieWriteStat writeStat2 = createWriteStat("222", "222base", Arrays.asList("3.log", "4.log")); + org.apache.hudi.avro.model.HoodieWriteStat writeStat3 = createWriteStat("333", null, Collections.singletonList("5.log")); + Map> partitionToWriteStatsMap = new HashMap<>(); + partitionToWriteStatsMap.put("partition1", Arrays.asList(writeStat1, writeStat2)); + partitionToWriteStatsMap.put("partition2", Collections.singletonList(writeStat3)); + commitMetadata.setPartitionToWriteStats(partitionToWriteStatsMap); + byte[] serializedCommitMetadata = TimelineMetadataUtils.serializeAvroMetadata( + commitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class).get(); + // Case: Reading 1.x written commit metadata + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "1"); + org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata1 = + COMMIT_METADATA_SER_DE.deserialize(instant, + serializedCommitMetadata, org.apache.hudi.common.model.HoodieCommitMetadata.class); + assertEquals(2, commitMetadata1.partitionToWriteStats.size()); + assertEquals(2, commitMetadata1.partitionToWriteStats.get("partition1").size()); + assertEquals(2, commitMetadata1.partitionToWriteStats.get("partition1").size()); + assertEquals("111", commitMetadata1.partitionToWriteStats.get("partition1").get(0).getFileId()); + assertEquals("222", commitMetadata1.partitionToWriteStats.get("partition1").get(1).getFileId()); + assertEquals("333", commitMetadata1.partitionToWriteStats.get("partition2").get(0).getFileId()); + + // Case: Reading 0.x written commit metadata + HoodieInstant legacyInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "1", "1",true); + CommitMetadataSerDe v1SerDe = new CommitMetadataSerDeV1(); + byte[] v1Bytes = v1SerDe.serialize(commitMetadata1).get(); + System.out.println(new String(v1Bytes)); + org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata2 = + COMMIT_METADATA_SER_DE.deserialize(legacyInstant, v1Bytes, org.apache.hudi.common.model.HoodieCommitMetadata.class); + assertEquals(2, commitMetadata2.partitionToWriteStats.size()); + assertEquals(2, commitMetadata2.partitionToWriteStats.get("partition1").size()); + assertEquals(2, commitMetadata2.partitionToWriteStats.get("partition1").size()); + System.out.println(commitMetadata2.partitionToWriteStats.get("partition1").get(0)); + assertEquals("111", commitMetadata2.partitionToWriteStats.get("partition1").get(0).getFileId()); + assertEquals("222", commitMetadata2.partitionToWriteStats.get("partition1").get(1).getFileId()); + assertEquals("333", commitMetadata2.partitionToWriteStats.get("partition2").get(0).getFileId()); + } + private org.apache.hudi.avro.model.HoodieWriteStat createWriteStat(String fileId, String baseFile, List logFiles) { org.apache.hudi.avro.model.HoodieWriteStat writeStat = new org.apache.hudi.avro.model.HoodieWriteStat(); writeStat.setFileId(fileId); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java index e9ec03efdc21..a99e48482990 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieWriteStat.java @@ -20,7 +20,7 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableConfig; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.storage.StoragePath; import org.junit.jupiter.api.Test; @@ -38,7 +38,7 @@ public class TestHoodieWriteStat { @Test public void testSetPaths() { - String instantTime = HoodieActiveTimeline.formatDate(new Date()); + String instantTime = TimelineUtils.formatDate(new Date()); String basePathString = "/data/tables/some-hoodie-table"; String partitionPathString = "2017/12/31"; String fileName = UUID.randomUUID().toString(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java index 32ec2c5f828f..ef3a04398b6b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/read/TestHoodieFileGroupReaderBase.java @@ -219,7 +219,7 @@ private void validateOutputFromFileGroupReader(StorageConfiguration storageCo getHoodieReaderContext(tablePath, avroSchema, storageConf), metaClient.getStorage(), tablePath, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), + metaClient.getActiveTimeline().lastInstant().get().requestedTime(), fileSlice, avroSchema, avroSchema, @@ -234,7 +234,7 @@ private void validateOutputFromFileGroupReader(StorageConfiguration storageCo getHoodieReaderContext(tablePath, avroSchema, storageConf), metaClient.getStorage(), tablePath, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), + metaClient.getActiveTimeline().lastInstant().get().requestedTime(), fileSlice, avroSchema, avroSchema, @@ -259,7 +259,7 @@ private void validateOutputFromFileGroupReader(StorageConfiguration storageCo getHoodieReaderContext(tablePath, avroSchema, storageConf), metaClient.getStorage(), tablePath, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), + metaClient.getActiveTimeline().lastInstant().get().requestedTime(), fileSlice, avroSchema, avroSchema, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestTimelineLayout.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestTimelineLayout.java index 4a5023884995..a081908fbe38 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestTimelineLayout.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestTimelineLayout.java @@ -27,6 +27,7 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -38,44 +39,44 @@ public class TestTimelineLayout { @Test public void testTimelineLayoutFilter() { List rawInstants = Arrays.asList( - new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "001"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "001"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "003"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "004"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"), - new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "007"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007")); + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "001"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "001"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "003"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "004"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, "007"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007")); - List layout0Instants = TimelineLayout.getLayout(new TimelineLayoutVersion(0)) + List layout0Instants = TimelineLayout.fromVersion(new TimelineLayoutVersion(0)) .filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList()); assertEquals(rawInstants, layout0Instants); - List layout1Instants = TimelineLayout.getLayout(TimelineLayoutVersion.CURR_LAYOUT_VERSION) + List layout1Instants = TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION) .filterHoodieInstants(rawInstants.stream()).collect(Collectors.toList()); assertEquals(7, layout1Instants.size()); assertTrue(layout1Instants.contains( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007"))); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "007"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"))); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "006"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "005"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"))); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "004"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002"))); assertTrue(layout1Instants.contains( - new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"))); + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, "001"))); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java index 9ca9d88f874f..1bf9c8a825ff 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestPriorityBasedFileSystemView.java @@ -26,6 +26,7 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.MockHoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ImmutablePair; @@ -697,7 +698,7 @@ public void testReset_withSecondaryInitialized() { @Test public void testGetLastInstant() { Option actual; - Option expected = Option.of(new HoodieInstant(true, "", "")); + Option expected = Option.of(HoodieTestUtils.INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, "", "")); when(primary.getLastInstant()).thenReturn(expected); actual = fsView.getLastInstant(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java index 01f2bc1ed88f..2bc8f7fba627 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/CompactionTestUtils.java @@ -53,6 +53,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createLogFile; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createMetaClient; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNull; @@ -110,7 +111,7 @@ public static Map> se } }); - metaClient = createMetaClient(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); + metaClient = createMetaClient(metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion()); Map> pendingCompactionMap = CompactionUtils.getAllPendingCompactionOperations(metaClient); @@ -137,23 +138,23 @@ public static Map> ge public static void scheduleCompaction(HoodieTableMetaClient metaClient, String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { metaClient.getActiveTimeline().saveToCompactionRequested( - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, COMPACTION_ACTION, instantTime), TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); } public static void createDeltaCommit(HoodieTableMetaClient metaClient, String instantTime) { - HoodieInstant requested = new HoodieInstant(State.REQUESTED, DELTA_COMMIT_ACTION, instantTime); + HoodieInstant requested = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, DELTA_COMMIT_ACTION, instantTime); metaClient.getActiveTimeline().createNewInstant(requested); metaClient.getActiveTimeline().transitionRequestedToInflight(requested, Option.empty()); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty()); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, DELTA_COMMIT_ACTION, instantTime), Option.empty()); } public static void scheduleInflightCompaction(HoodieTableMetaClient metaClient, String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { scheduleCompaction(metaClient, instantTime, compactionPlan); metaClient.getActiveTimeline() - .transitionCompactionRequestedToInflight(new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime)); + .transitionCompactionRequestedToInflight(INSTANT_GENERATOR.createNewInstant(State.REQUESTED, COMPACTION_ACTION, instantTime)); } public static HoodieCompactionPlan createCompactionPlan(HoodieTableMetaClient metaClient, String instantTime, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index fc36397d9961..12b8ee19ac83 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -36,6 +36,7 @@ import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; @@ -205,16 +206,18 @@ public static void createCommit(String basePath, String instantTime) throws IOEx createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } - public static void createCommit(String basePath, String instantTime, Option metadata) throws IOException { - createCommit(basePath, instantTime, Option.empty(), metadata); + public static void createCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, String instantTime, + Option metadata) throws IOException { + createCommit(commitMetadataSerDe, basePath, instantTime, Option.empty(), metadata); } - public static void createCommit(String basePath, String instantTime, Option completionTime, Option metadata) throws IOException { + public static void createCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, String instantTime, + Option completionTime, Option metadata) throws IOException { final Supplier completionTimeSupplier = () -> completionTime.isPresent() ? completionTime.get() : InProcessTimeGenerator.createNewInstantTime(); if (metadata.isPresent()) { HoodieCommitMetadata commitMetadata = metadata.get(); createMetaFile(basePath, instantTime, completionTimeSupplier, HoodieTimeline.COMMIT_EXTENSION, - serializeCommitMetadata(commitMetadata).get()); + serializeCommitMetadata(commitMetadataSerDe, commitMetadata).get()); } else { createMetaFile(basePath, instantTime, completionTimeSupplier, HoodieTimeline.COMMIT_EXTENSION, getUTF8Bytes("")); @@ -241,10 +244,10 @@ public static void createInflightCommit(String basePath, String instantTime) thr createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMMIT_EXTENSION); } - public static void createDeltaCommit(String basePath, String instantTime, + public static void createDeltaCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, String instantTime, HoodieCommitMetadata metadata) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.DELTA_COMMIT_EXTENSION, - serializeCommitMetadata(metadata).get()); + serializeCommitMetadata(commitMetadataSerDe, metadata).get()); } public static void createDeltaCommit(String basePath, String instantTime) throws IOException { @@ -271,10 +274,10 @@ public static void createInflightReplaceCommit(String basePath, String instantTi createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); } - public static void createReplaceCommit(String basePath, String instantTime, - HoodieReplaceCommitMetadata metadata) throws IOException { + public static void createReplaceCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, + String instantTime, HoodieReplaceCommitMetadata metadata) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, - serializeCommitMetadata(metadata).get()); + serializeCommitMetadata(commitMetadataSerDe, metadata).get()); } public static void createRequestedClusterCommit(String basePath, String instantTime, @@ -284,12 +287,12 @@ public static void createRequestedClusterCommit(String basePath, String instantT serializeRequestedReplaceMetadata(requestedReplaceMetadata).get()); } - public static void createInflightClusterCommit(String basePath, String instantTime, - Option inflightReplaceMetadata) + public static void createInflightClusterCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, + String instantTime, Option inflightReplaceMetadata) throws IOException { if (inflightReplaceMetadata.isPresent()) { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLUSTERING_COMMIT_EXTENSION, - serializeCommitMetadata(inflightReplaceMetadata.get()).get()); + serializeCommitMetadata(commitMetadataSerDe, inflightReplaceMetadata.get()).get()); } else { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_CLUSTERING_COMMIT_EXTENSION); } @@ -306,12 +309,12 @@ public static void createRequestedReplaceCommit(String basePath, String instantT } } - public static void createInflightReplaceCommit(String basePath, String instantTime, - Option inflightReplaceMetadata) + public static void createInflightReplaceCommit(CommitMetadataSerDe commitMetadataSerDe, String basePath, + String instantTime, Option inflightReplaceMetadata) throws IOException { if (inflightReplaceMetadata.isPresent()) { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION, - serializeCommitMetadata(inflightReplaceMetadata.get()).get()); + serializeCommitMetadata(commitMetadataSerDe, inflightReplaceMetadata.get()).get()); } else { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 554c3e0791b6..b075248cf488 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -30,7 +30,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; @@ -79,6 +78,8 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -166,7 +167,6 @@ public class HoodieTestDataGenerator implements AutoCloseable { public static final Schema AVRO_SHORT_TRIP_SCHEMA = new Schema.Parser().parse(SHORT_TRIP_SCHEMA); public static final Schema AVRO_TRIP_SCHEMA = new Schema.Parser().parse(TRIP_SCHEMA); public static final Schema FLATTENED_AVRO_SCHEMA = new Schema.Parser().parse(TRIP_FLATTENED_SCHEMA); - private final Random rand; //Maintains all the existing keys schema wise @@ -530,13 +530,13 @@ public GenericRecord generateRecordForShortTripSchema(String rowKey, String ride public static void createRequestedCommitFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path pendingRequestedFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeRequestedCommitFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeRequestedCommitFileName(instantTime)); createEmptyFile(basePath, pendingRequestedFile, configuration); } public static void createPendingCommitFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path pendingCommitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeInflightCommitFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeInflightCommitFileName(instantTime)); createEmptyFile(basePath, pendingCommitFile, configuration); } @@ -546,8 +546,9 @@ public static void createCommitFile(String basePath, String instantTime, Storage } private static void createCommitFile(String basePath, String instantTime, StorageConfiguration configuration, HoodieCommitMetadata commitMetadata) { - Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), HoodieTimeline.makeInflightCommitFileName(instantTime), - HoodieTimeline.makeRequestedCommitFileName(instantTime)) + Arrays.asList(INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + INSTANT_FILE_NAME_GENERATOR.makeInflightCommitFileName(instantTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedCommitFileName(instantTime)) .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); } @@ -557,7 +558,7 @@ public static void createOnlyCompletedCommitFile(String basePath, String instant } public static void createOnlyCompletedCommitFile(String basePath, String instantTime, StorageConfiguration configuration, HoodieCommitMetadata commitMetadata) { - createMetadataFile(HoodieTimeline.makeCommitFileName(instantTime), basePath, configuration, commitMetadata); + createMetadataFile(INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(instantTime), basePath, configuration, commitMetadata); } public static void createDeltaCommitFile(String basePath, String instantTime, StorageConfiguration configuration) { @@ -566,15 +567,15 @@ public static void createDeltaCommitFile(String basePath, String instantTime, St } private static void createDeltaCommitFile(String basePath, String instantTime, StorageConfiguration configuration, HoodieCommitMetadata commitMetadata) { - Arrays.asList(HoodieTimeline.makeDeltaFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - HoodieTimeline.makeInflightDeltaFileName(instantTime), - HoodieTimeline.makeRequestedDeltaFileName(instantTime)) + Arrays.asList(INSTANT_FILE_NAME_GENERATOR.makeDeltaFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + INSTANT_FILE_NAME_GENERATOR.makeInflightDeltaFileName(instantTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedDeltaFileName(instantTime)) .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); } private static void createMetadataFile(String f, String basePath, StorageConfiguration configuration, HoodieCommitMetadata commitMetadata) { try { - createMetadataFile(f, basePath, configuration, serializeCommitMetadata(commitMetadata).get()); + createMetadataFile(f, basePath, configuration, serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } catch (IOException e) { throw new HoodieIOException(e.getMessage(), e); } @@ -605,20 +606,20 @@ private static void createMetadataFile(String f, String basePath, StorageConfigu public static void createReplaceCommitRequestedFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeRequestedReplaceFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeRequestedReplaceFileName(instantTime)); createEmptyFile(basePath, commitFile, configuration); } public static void createReplaceCommitInflightFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeInflightReplaceFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeInflightReplaceFileName(instantTime)); createEmptyFile(basePath, commitFile, configuration); } private static void createPendingClusterFile(String basePath, String instantTime, StorageConfiguration configuration, HoodieCommitMetadata commitMetadata) { - Arrays.asList(HoodieTimeline.makeInflightClusteringFileName(instantTime), - HoodieTimeline.makeRequestedClusteringFileName(instantTime)) + Arrays.asList(INSTANT_FILE_NAME_GENERATOR.makeInflightClusteringFileName(instantTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedClusteringFileName(instantTime)) .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); } @@ -630,7 +631,7 @@ public static void createPendingClusterFile(String basePath, String instantTime, public static void createEmptyCleanRequestedFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeRequestedCleanerFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeRequestedCleanerFileName(instantTime)); createEmptyFile(basePath, commitFile, configuration); } @@ -643,14 +644,14 @@ private static void createEmptyFile(String basePath, Path filePath, StorageConfi public static void createCompactionRequestedFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeRequestedCompactionFileName(instantTime)); + + INSTANT_FILE_NAME_GENERATOR.makeRequestedCompactionFileName(instantTime)); createEmptyFile(basePath, commitFile, configuration); } public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInstant instant, StorageConfiguration configuration) throws IOException { Path commitFile = - new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName()); + new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + INSTANT_FILE_NAME_GENERATOR.getFileName(instant)); HoodieStorage storage = HoodieStorageUtils.getStorage(basePath, configuration); try (OutputStream os = storage.create(new StoragePath(commitFile.toUri()), true)) { HoodieCompactionPlan workload = HoodieCompactionPlan.newBuilder().setVersion(1).build(); @@ -662,12 +663,12 @@ public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInst public static void createSavepointFile(String basePath, String instantTime, StorageConfiguration configuration) throws IOException { Path commitFile = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeSavePointFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime())); + + INSTANT_FILE_NAME_GENERATOR.makeSavePointFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime())); HoodieStorage storage = HoodieStorageUtils.getStorage(basePath, configuration); try (OutputStream os = storage.create(new StoragePath(commitFile.toUri()), true)) { HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); // Write empty commit metadata - os.write(serializeCommitMetadata(commitMetadata).get()); + os.write(serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index ec8a6c8cb316..9399df0f1ab9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -28,8 +28,19 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; +import org.apache.hudi.common.table.timeline.CommitMetadataSerDe; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.DefaultCommitMetadataSerDe; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantFileNameParser; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantGenerator; +import org.apache.hudi.common.table.timeline.versioning.DefaultTimelineFactory; import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameParser; +import org.apache.hudi.common.table.timeline.TimelineFactory; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; @@ -73,6 +84,11 @@ public class HoodieTestUtils { public static final int DEFAULT_LOG_VERSION = 1; public static final String[] DEFAULT_PARTITION_PATHS = {"2016/03/15", "2015/03/16", "2015/03/17"}; public static final String HADOOP_STORAGE_CONF = "org.apache.hudi.storage.hadoop.HadoopStorageConfiguration"; + public static final InstantGenerator INSTANT_GENERATOR = new DefaultInstantGenerator(); + public static final TimelineFactory TIMELINE_FACTORY = new DefaultTimelineFactory(); + public static final InstantFileNameGenerator INSTANT_FILE_NAME_GENERATOR = new DefaultInstantFileNameGenerator(); + public static final InstantFileNameParser INSTANT_FILE_NAME_PARSER = new DefaultInstantFileNameParser(); + public static final CommitMetadataSerDe COMMIT_METADATA_SER_DE = new DefaultCommitMetadataSerDe(); public static StorageConfiguration getDefaultStorageConf() { return (StorageConfiguration) ReflectionUtils.loadClass(HADOOP_STORAGE_CONF, @@ -235,8 +251,8 @@ public static HoodieTableMetaClient createMetaClient(StorageConfiguration sto } public static HoodieTableMetaClient createMetaClient(StorageConfiguration storageConf, - StoragePath basePath) { - return HoodieTableMetaClient.builder() + StoragePath basePath, HoodieTableVersion tableVersion) { + return HoodieTableMetaClient.builder().setLayoutVersion(Option.of(tableVersion.getTimelineLayoutVersion())) .setConf(storageConf).setBasePath(basePath).build(); } @@ -352,7 +368,7 @@ public static List getLogFileListFromFileSlice(FileSlice fileSlice) { public static HoodieInstant getCompleteInstant(HoodieStorage storage, StoragePath parent, String instantTime, String action) { - return new HoodieInstant(getCompleteInstantFileInfo(storage, parent, instantTime, action)); + return INSTANT_GENERATOR.createNewInstant(getCompleteInstantFileInfo(storage, parent, instantTime, action)); } public static StoragePath getCompleteInstantPath(HoodieStorage storage, StoragePath parent, diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/InProcessTimeGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/InProcessTimeGenerator.java index 265047d07479..aa3b01e19edb 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/InProcessTimeGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/InProcessTimeGenerator.java @@ -20,7 +20,7 @@ package org.apache.hudi.common.testutils; import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; @@ -40,6 +40,6 @@ public static String createNewInstantTime() { public static String createNewInstantTime(long milliseconds) { // We don't lock here since many callers are in hudi-common, which doesn't contain InProcessLockProvider - return HoodieActiveTimeline.createNewInstantTime(false, TIME_GENERATOR, milliseconds); + return HoodieInstantTimeGenerator.createNewInstantTime(false, TIME_GENERATOR, milliseconds); } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/MockHoodieTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/MockHoodieTimeline.java index 01c417ad111f..827964e727b4 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/MockHoodieTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/MockHoodieTimeline.java @@ -21,6 +21,11 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantGenerator; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; import java.util.Comparator; import java.util.List; @@ -30,15 +35,18 @@ /** * A mocked {@link HoodieActiveTimeline}. */ -public class MockHoodieTimeline extends HoodieActiveTimeline { +public class MockHoodieTimeline extends ActiveTimelineV2 { public MockHoodieTimeline(Stream completed, Stream inflights) { super(); + InstantGenerator instantGenerator = new DefaultInstantGenerator(); + InstantFileNameGenerator instantFileNameGenerator = new DefaultInstantFileNameGenerator(); + this.setInstants(Stream - .concat(completed.map(s -> new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, s, + .concat(completed.map(s -> instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, s, InProcessTimeGenerator.createNewInstantTime())), - inflights.map(s -> new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, s))) - .sorted(Comparator.comparing(HoodieInstant::getFileName)).collect(Collectors.toList())); + inflights.map(s -> instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, s))) + .sorted(Comparator.comparing(instantFileNameGenerator::getFileName)).collect(Collectors.toList())); } public MockHoodieTimeline(List instants) { diff --git a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/common/RandomJsonSource.java b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/common/RandomJsonSource.java index c2b876c7ff12..6fc24d7418d3 100644 --- a/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/common/RandomJsonSource.java +++ b/hudi-examples/hudi-examples-spark/src/main/java/org/apache/hudi/examples/common/RandomJsonSource.java @@ -21,9 +21,9 @@ import org.apache.hudi.common.config.HoodieTimeGeneratorConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.utilities.schema.SchemaProvider; @@ -49,7 +49,7 @@ public RandomJsonSource(TypedProperties props, JavaSparkContext sparkContext, Sp } protected InputBatch> fetchNewData(Option lastCkptStr, long sourceLimit) { - String commitTime = HoodieActiveTimeline.createNewInstantTime(true, timeGenerator); + String commitTime = TimelineUtils.generateInstantTime(true, timeGenerator); List inserts = dataGen.convertToStringList(dataGen.generateInserts(commitTime, 20)); return new InputBatch<>(Option.of(sparkContext.parallelize(inserts, 1)), commitTime); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkClusteringMetrics.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkClusteringMetrics.java index 081c8f79a73f..fa82d8192365 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkClusteringMetrics.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkClusteringMetrics.java @@ -86,7 +86,7 @@ public void setFirstPendingClusteringInstant(Option firstPendingC if (!firstPendingClusteringInstant.isPresent()) { this.clusteringDelay = 0L; } else { - Instant start = HoodieInstantTimeGenerator.parseDateFromInstantTime((firstPendingClusteringInstant.get()).getTimestamp()).toInstant(); + Instant start = HoodieInstantTimeGenerator.parseDateFromInstantTime((firstPendingClusteringInstant.get()).requestedTime()).toInstant(); this.clusteringDelay = Duration.between(start, Instant.now()).getSeconds(); } } catch (ParseException e) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkCompactionMetrics.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkCompactionMetrics.java index 20ad3682f4d5..eaf9e52bfa9c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkCompactionMetrics.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/metrics/FlinkCompactionMetrics.java @@ -94,7 +94,7 @@ public void setFirstPendingCompactionInstant(Option firstPendingC if (!firstPendingCompactionInstant.isPresent()) { this.compactionDelay = 0L; } else { - Instant start = HoodieInstantTimeGenerator.parseDateFromInstantTime(firstPendingCompactionInstant.get().getTimestamp()).toInstant(); + Instant start = HoodieInstantTimeGenerator.parseDateFromInstantTime(firstPendingCompactionInstant.get().requestedTime()).toInstant(); this.compactionDelay = Duration.between(start, Instant.now()).getSeconds(); } } catch (ParseException e) { diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java index 207c301e425d..fbb2a5ff391c 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/StreamWriteOperatorCoordinator.java @@ -72,6 +72,8 @@ import java.util.concurrent.CompletionException; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.util.StreamerUtil.initTableIfNotExists; /** @@ -487,7 +489,7 @@ private void handleWriteMetaEvent(WriteMetadataEvent event) { // if it checkpoints succeed then flushes the data buffer again before this coordinator receives a checkpoint // success event, the data buffer would flush with an older instant time. ValidationUtils.checkState( - HoodieTimeline.compareTimestamps(this.instant, HoodieTimeline.GREATER_THAN_OR_EQUALS, event.getInstantTime()), + compareTimestamps(this.instant, GREATER_THAN_OR_EQUALS, event.getInstantTime()), String.format("Receive an unexpected event for instant %s from task %d", event.getInstantTime(), event.getTaskID())); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java index 86c6ee39f2c4..71fc1e1e33f9 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bootstrap/BootstrapOperator.java @@ -209,7 +209,7 @@ protected void loadRecords(String partitionPath) throws Exception { Schema schema = new TableSchemaResolver(this.hoodieTable.getMetaClient()).getTableAvroSchema(); List fileSlices = this.hoodieTable.getSliceView() - .getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().getTimestamp()) + .getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.get().requestedTime()) .collect(toList()); for (FileSlice fileSlice : fileSlices) { @@ -240,7 +240,7 @@ protected void loadRecords(String partitionPath) throws Exception { .map(logFile -> logFile.getPath().toString()) .collect(toList()); - try (HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(logPaths, schema, latestCommitTime.get().getTimestamp(), + try (HoodieMergedLogRecordScanner scanner = FormatUtils.logScanner(logPaths, schema, latestCommitTime.get().requestedTime(), writeConfig, hadoopConf)) { for (String recordKey : scanner.getRecords().keySet()) { output.collect(new StreamRecord(new IndexRecord(generateHoodieRecord(new HoodieKey(recordKey, partitionPath), fileSlice)))); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignFunction.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignFunction.java index d51d59402f50..125a2161e5bd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/bucket/ConsistentBucketAssignFunction.java @@ -145,7 +145,7 @@ public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throw LOG.info("Clear up cached hashing metadata because find a new replace commit.\n Instant: {}.\n Effected Partitions: {}.", lastRefreshInstant, affectedPartitions); affectedPartitions.forEach(this.partitionToIdentifier::remove); } - this.lastRefreshInstant = timeline.lastInstant().get().getTimestamp(); + this.lastRefreshInstant = timeline.lastInstant().get().requestedTime(); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java index eea30abf53b9..e773f3ed1171 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringCommitSink.java @@ -143,7 +143,7 @@ private void commitIfNecessary(String instant, Collection HoodieClusteringPlan clusteringPlan = clusteringPlanCache.computeIfAbsent(instant, k -> { try { HoodieTableMetaClient metaClient = this.writeClient.getHoodieTable().getMetaClient(); - return ClusteringUtils.getInflightClusteringInstant(instant, metaClient.getActiveTimeline()) + return ClusteringUtils.getInflightClusteringInstant(instant, metaClient.getActiveTimeline(), table.getInstantGenerator()) .flatMap(pendingInstant -> ClusteringUtils.getClusteringPlan( metaClient, pendingInstant)) .map(Pair::getRight) diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java index 8dfe83302d48..2bba6b9160dd 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/ClusteringPlanOperator.java @@ -114,7 +114,7 @@ private void scheduleClustering(HoodieFlinkTable table, long checkpointId) { return; } - String clusteringInstantTime = firstRequested.get().getTimestamp(); + String clusteringInstantTime = firstRequested.get().requestedTime(); // generate clustering plan // should support configurable commit metadata diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java index f6fc148c39d1..e61f140bc50a 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/HoodieFlinkClusteringJob.java @@ -270,7 +270,7 @@ private void cluster() throws Exception { final HoodieInstant clusteringInstant; if (cfg.clusteringInstantTime != null) { clusteringInstant = instants.stream() - .filter(i -> i.getTimestamp().equals(cfg.clusteringInstantTime)) + .filter(i -> i.requestedTime().equals(cfg.clusteringInstantTime)) .findFirst() .orElseThrow(() -> new HoodieException("Clustering instant [" + cfg.clusteringInstantTime + "] not found")); } else { @@ -279,7 +279,8 @@ private void cluster() throws Exception { CompactionUtil.isLIFO(cfg.clusteringSeq) ? instants.get(instants.size() - 1) : instants.get(0); } - Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant.getTimestamp(), table.getActiveTimeline()); + Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(clusteringInstant.requestedTime(), + table.getActiveTimeline(), table.getInstantGenerator()); if (inflightInstantOpt.isPresent()) { LOG.info("Rollback inflight clustering instant: [" + clusteringInstant + "]"); table.rollbackInflightClustering(inflightInstantOpt.get(), @@ -303,11 +304,12 @@ private void cluster() throws Exception { if (clusteringPlan == null || (clusteringPlan.getInputGroups() == null) || (clusteringPlan.getInputGroups().isEmpty())) { // no clustering plan, do nothing and return. - LOG.info("No clustering plan for instant " + clusteringInstant.getTimestamp()); + LOG.info("No clustering plan for instant " + clusteringInstant.requestedTime()); return; } - HoodieInstant instant = ClusteringUtils.getRequestedClusteringInstant(clusteringInstant.getTimestamp(), table.getActiveTimeline()).get(); + HoodieInstant instant = ClusteringUtils.getRequestedClusteringInstant(clusteringInstant.requestedTime(), + table.getActiveTimeline(), table.getInstantGenerator()).get(); int inputGroupSize = clusteringPlan.getInputGroups().size(); @@ -329,7 +331,7 @@ private void cluster() throws Exception { long ckpTimeout = env.getCheckpointConfig().getCheckpointTimeout(); conf.setLong(FlinkOptions.WRITE_COMMIT_ACK_TIMEOUT, ckpTimeout); - DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(clusteringInstant.getTimestamp(), clusteringPlan, conf)) + DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(clusteringInstant.requestedTime(), clusteringPlan, conf)) .name("clustering_source") .uid("uid_clustering_source") .rebalance() @@ -351,7 +353,7 @@ private void cluster() throws Exception { .getTransformation() .setMaxParallelism(1); - env.execute("flink_hudi_clustering_" + clusteringInstant.getTimestamp()); + env.execute("flink_hudi_clustering_" + clusteringInstant.requestedTime()); } /** diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/update/strategy/FlinkConsistentBucketUpdateStrategy.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/update/strategy/FlinkConsistentBucketUpdateStrategy.java index f9a58a7318d5..73c0d655fb72 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/update/strategy/FlinkConsistentBucketUpdateStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/clustering/update/strategy/FlinkConsistentBucketUpdateStrategy.java @@ -75,13 +75,13 @@ public void initialize(HoodieFlinkWriteClient writeClient) { List instants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()); if (!instants.isEmpty()) { HoodieInstant latestPendingReplaceInstant = instants.get(instants.size() - 1); - if (latestPendingReplaceInstant.getTimestamp().compareTo(lastRefreshInstant) > 0) { + if (latestPendingReplaceInstant.requestedTime().compareTo(lastRefreshInstant) > 0) { LOG.info("Found new pending replacement commit. Last pending replacement commit is {}.", latestPendingReplaceInstant); this.table = table; this.fileGroupsInPendingClustering = table.getFileSystemView().getFileGroupsInPendingClustering() .map(Pair::getKey).collect(Collectors.toSet()); // TODO throw exception if exists bucket merge plan - this.lastRefreshInstant = latestPendingReplaceInstant.getTimestamp(); + this.lastRefreshInstant = latestPendingReplaceInstant.requestedTime(); this.partitionToIdentifier.clear(); } } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java index 91da3a48f641..75f5779cb9ec 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/CompactionPlanOperator.java @@ -125,7 +125,7 @@ private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) th return; } - String compactionInstantTime = firstRequested.get().getTimestamp(); + String compactionInstantTime = firstRequested.get().requestedTime(); // generate compaction plan // should support configurable commit metadata @@ -137,7 +137,7 @@ private void scheduleCompaction(HoodieFlinkTable table, long checkpointId) th // do nothing. LOG.info("Empty compaction plan for instant " + compactionInstantTime); } else { - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant instant = table.getInstantGenerator().getCompactionRequestedInstant(compactionInstantTime); // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java index 99dd45d94b49..fb23207604a7 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/compact/HoodieFlinkCompactor.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -239,9 +240,9 @@ private void compact() throws Exception { return; } - List compactionInstantTimes = requested.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List compactionInstantTimes = requested.stream().map(HoodieInstant::requestedTime).collect(Collectors.toList()); compactionInstantTimes.forEach(timestamp -> { - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(timestamp); + HoodieInstant inflightInstant = table.getInstantGenerator().getCompactionInflightInstant(timestamp); if (pendingCompactionTimeline.containsInstant(inflightInstant)) { LOG.info("Rollback inflight compaction instant: [" + timestamp + "]"); table.rollbackInflightCompaction(inflightInstant); @@ -269,7 +270,8 @@ private void compact() throws Exception { return; } - List instants = compactionInstantTimes.stream().map(HoodieTimeline::getCompactionRequestedInstant).collect(Collectors.toList()); + InstantGenerator instantGenerator = table.getInstantGenerator(); + List instants = compactionInstantTimes.stream().map(instantGenerator::getCompactionRequestedInstant).collect(Collectors.toList()); int totalOperations = Math.toIntExact(compactionPlans.stream().mapToLong(pair -> pair.getRight().getOperations().size()).sum()); 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 5c9a1a7b3351..222a4d8ba2b7 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 @@ -54,7 +54,7 @@ public static CompactionPlanStrategy getStrategy(FlinkCompactionConfig config) { } List instants = Arrays.asList(config.compactionPlanInstant.split(",")); return pendingCompactionTimeline.getInstantsAsStream() - .filter(instant -> instants.contains(instant.getTimestamp())) + .filter(instant -> instants.contains(instant.requestedTime())) .collect(Collectors.toList()); }; case CompactionPlanStrategy.NUM_INSTANTS: diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java index 14f78d1f1bd0..7286bda45b50 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/DeltaWriteProfile.java @@ -59,7 +59,7 @@ protected List smallFilesProfile(String partitionPath) { List allSmallFileSlices = new ArrayList<>(); // If we can index log files, we can add more inserts to log files for fileIds including those under // pending compaction. - List allFileSlices = fsView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()) + List allFileSlices = fsView.getLatestMergedFileSlicesBeforeOrOn(partitionPath, latestCommitTime.requestedTime()) .collect(Collectors.toList()); for (FileSlice fileSlice : allFileSlices) { if (isSmallFile(fileSlice)) { 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 020c18044c81..9e06191bae59 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 @@ -154,7 +154,7 @@ private long averageBytesPerRecord() { HoodieInstant instant = instants.next(); final HoodieCommitMetadata commitMetadata = this.metadataCache.computeIfAbsent( - instant.getTimestamp(), + instant.requestedTime(), k -> WriteProfiles.getCommitMetadataSafely(config.getTableName(), basePath, instant, commitTimeline) .orElse(null)); if (commitMetadata == null) { @@ -206,7 +206,7 @@ protected List smallFilesProfile(String partitionPath) { if (!commitTimeline.empty()) { // if we have some commits HoodieInstant latestCommitTime = commitTimeline.lastInstant().get(); List allFiles = fsView - .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.getTimestamp()).collect(Collectors.toList()); + .getLatestBaseFilesBeforeOrOn(partitionPath, latestCommitTime.requestedTime()).collect(Collectors.toList()); for (HoodieBaseFile file : allFiles) { // filter out the corrupted files. @@ -231,7 +231,7 @@ protected SyncableFileSystemView getFileSystemView() { * whose instant does not belong to the given instants {@code instants}. */ private void cleanMetadataCache(Stream instants) { - Set timestampSet = instants.map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + Set timestampSet = instants.map(HoodieInstant::requestedTime).collect(Collectors.toSet()); this.metadataCache.keySet().retainAll(timestampSet); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java index b2c7d56f6245..c11d54239000 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/profile/WriteProfiles.java @@ -23,6 +23,7 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; @@ -173,14 +174,15 @@ public static Option getCommitMetadataSafely( HoodieTimeline timeline) { try { byte[] data = timeline.getInstantDetails(instant).get(); - return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class)); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + return Option.of(layout.getCommitMetadataSerDe().deserialize(instant, data, HoodieCommitMetadata.class)); } catch (FileNotFoundException fe) { // make this fail safe. - LOG.warn("Instant {} was deleted by the cleaner, ignore", instant.getTimestamp()); + LOG.warn("Instant {} was deleted by the cleaner, ignore", instant.requestedTime()); return Option.empty(); } catch (Throwable throwable) { LOG.error("Get write metadata for table {} with instant {} and path: {} error", - tableName, instant.getTimestamp(), basePath); + tableName, instant.requestedTime(), basePath); return Option.empty(); } } @@ -203,7 +205,7 @@ public static HoodieCommitMetadata getCommitMetadata( return TimelineUtils.getCommitMetadata(instant, timeline); } catch (IOException e) { LOG.error("Get write metadata for table {} with instant {} and path: {} error", - tableName, instant.getTimestamp(), basePath); + tableName, instant.requestedTime(), basePath); throw new HoodieException(e); } } 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 f09cca2e5f5a..60bef1f3a929 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 @@ -31,6 +31,7 @@ import org.apache.hudi.common.table.read.IncrementalQueryAnalyzer; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.util.Option; import org.apache.hudi.configuration.FlinkOptions; @@ -372,7 +373,7 @@ private List getInputSplits( String basePath = fileSlice.getBaseFile().map(BaseFile::getPath).orElse(null); // the latest commit is used as the limit of the log reader instant upper threshold, // it must be at least the latest instant time of the file slice to avoid data loss. - String latestCommit = HoodieTimeline.minInstant(fileSlice.getLatestInstantTime(), endInstant); + String latestCommit = InstantComparison.minInstant(fileSlice.getLatestInstantTime(), endInstant); return new MergeOnReadInputSplit(cnt.getAndAdd(1), basePath, logPaths, latestCommit, metaClient.getBasePath().toString(), maxCompactionMemoryInBytes, mergeType, instantRange, fileSlice.getFileId()); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java index 18710678bf5e..90f81289bd2b 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java @@ -399,7 +399,7 @@ private List buildInputSplits() { if (!fsView.getLastInstant().isPresent()) { return Collections.emptyList(); } - String latestCommit = fsView.getLastInstant().get().getTimestamp(); + String latestCommit = fsView.getLastInstant().get().requestedTime(); final String mergeType = this.conf.getString(FlinkOptions.MERGE_TYPE); final AtomicInteger cnt = new AtomicInteger(0); // generates one input split for each file group diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java index 139f630862ab..cdc489e938ca 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/InternalSchemaManager.java @@ -21,7 +21,9 @@ import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; -import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.InstantFileNameGenerator; +import org.apache.hudi.common.table.timeline.TimelineLayout; +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.util.InternalSchemaCache; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -58,12 +60,14 @@ public class InternalSchemaManager implements Serializable { private static final long serialVersionUID = 1L; - public static final InternalSchemaManager DISABLED = new InternalSchemaManager(null, InternalSchema.getEmptyInternalSchema(), null, null); + public static final InternalSchemaManager DISABLED = new InternalSchemaManager(null, InternalSchema.getEmptyInternalSchema(), null, null, + TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION)); private final Configuration conf; private final InternalSchema querySchema; private final String validCommits; private final String tablePath; + private final TimelineLayout layout; private transient org.apache.hadoop.conf.Configuration hadoopConf; public static InternalSchemaManager get(Configuration conf, HoodieTableMetaClient metaClient) { @@ -74,20 +78,24 @@ public static InternalSchemaManager get(Configuration conf, HoodieTableMetaClien if (!internalSchema.isPresent() || internalSchema.get().isEmptySchema()) { return DISABLED; } + + InstantFileNameGenerator factory = metaClient.getInstantFileNameGenerator(); String validCommits = metaClient .getCommitsAndCompactionTimeline() .filterCompletedInstants() .getInstantsAsStream() - .map(HoodieInstant::getFileName) + .map(factory::getFileName) .collect(Collectors.joining(",")); - return new InternalSchemaManager(conf, internalSchema.get(), validCommits, metaClient.getBasePath().toString()); + return new InternalSchemaManager(conf, internalSchema.get(), validCommits, metaClient.getBasePath().toString(), metaClient.getTimelineLayout()); } - public InternalSchemaManager(Configuration conf, InternalSchema querySchema, String validCommits, String tablePath) { + public InternalSchemaManager(Configuration conf, InternalSchema querySchema, String validCommits, String tablePath, + TimelineLayout layout) { this.conf = conf; this.querySchema = querySchema; this.validCommits = validCommits; this.tablePath = tablePath; + this.layout = layout; } public InternalSchema getQuerySchema() { @@ -113,7 +121,8 @@ InternalSchema getMergeSchema(String fileName) { InternalSchema fileSchema = InternalSchemaCache.getInternalSchemaByVersionId( commitInstantTime, tablePath, new HoodieHadoopStorage(tablePath, getHadoopConf()), - validCommits); + validCommits, layout.getInstantFileNameParser(), + layout.getCommitMetadataSerDe(), layout.getInstantGenerator()); if (querySchema.equals(fileSchema)) { return InternalSchema.getEmptyInternalSchema(); } diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClusteringUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClusteringUtil.java index 1c988e6d303e..1a154dfcd5af 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClusteringUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/ClusteringUtil.java @@ -105,8 +105,8 @@ public static void rollbackClustering(HoodieFlinkTable table, HoodieFlinkWrit */ public static void rollbackClustering(HoodieFlinkTable table, HoodieFlinkWriteClient writeClient, String instantTime) { HoodieActiveTimeline activeTimeline = table.getMetaClient().reloadActiveTimeline(); - Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(instantTime, activeTimeline); - if (inflightInstantOpt.isPresent() && ClusteringUtils.isClusteringInstant(activeTimeline, inflightInstantOpt.get())) { + Option inflightInstantOpt = ClusteringUtils.getInflightClusteringInstant(instantTime, activeTimeline, table.getInstantGenerator()); + if (inflightInstantOpt.isPresent() && ClusteringUtils.isClusteringInstant(activeTimeline, inflightInstantOpt.get(), table.getInstantGenerator())) { LOG.warn("Rollback failed clustering instant: [" + instantTime + "]"); table.rollbackInflightClustering(inflightInstantOpt.get(), commitToRollback -> writeClient.getTableServiceClient().getPendingRollbackInfo(table.getMetaClient(), commitToRollback, false)); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java index 133ec41dadcd..657bbdbea603 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/CompactionUtil.java @@ -137,7 +137,7 @@ public static void inferMetadataConf(Configuration conf, HoodieTableMetaClient m } public static void rollbackCompaction(HoodieFlinkTable table, String instantTime) { - HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(instantTime); + HoodieInstant inflightInstant = table.getInstantGenerator().getCompactionInflightInstant(instantTime); if (table.getMetaClient().reloadActiveTimeline().filterPendingCompactionTimeline().containsInstant(inflightInstant)) { LOG.warn("Rollback failed compaction instant: [" + instantTime + "]"); table.rollbackInflightCompaction(inflightInstant); @@ -177,7 +177,7 @@ public static void rollbackEarliestCompaction(HoodieFlinkTable table, Configu HoodieInstant instant = earliestInflight.get(); String currentTime = table.getMetaClient().createNewInstantTime(); int timeout = conf.getInteger(FlinkOptions.COMPACTION_TIMEOUT_SECONDS); - if (StreamerUtil.instantTimeDiffSeconds(currentTime, instant.getTimestamp()) >= timeout) { + if (StreamerUtil.instantTimeDiffSeconds(currentTime, instant.requestedTime()) >= timeout) { LOG.info("Rollback the inflight compaction instant: " + instant + " for timeout(" + timeout + "s)"); table.rollbackInflightCompaction(instant); table.getMetaClient().reloadActiveTimeline(); diff --git a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java index eadf42ca0511..db486c8738b5 100644 --- a/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/util/StreamerUtil.java @@ -31,9 +31,9 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.log.HoodieLogFormat; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; @@ -84,6 +84,9 @@ import static org.apache.hudi.common.model.HoodieFileFormat.ORC; import static org.apache.hudi.common.model.HoodieFileFormat.PARQUET; import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; /** * Utilities for Flink stream read and write. @@ -267,7 +270,6 @@ public static HoodieTableMetaClient initTableIfNotExists( .setUrlEncodePartitioning(conf.getBoolean(FlinkOptions.URL_ENCODE_PARTITIONING)) .setCDCEnabled(conf.getBoolean(FlinkOptions.CDC_ENABLED)) .setCDCSupplementalLoggingMode(conf.getString(FlinkOptions.SUPPLEMENTAL_LOGGING_MODE)) - .setTimelineLayoutVersion(1) .initTable(HadoopFSUtils.getStorageConfWithCopy(hadoopConf), basePath); LOG.info("Table initialized under base path {}", basePath); } else { @@ -386,16 +388,16 @@ public static Option getTableConfig(String basePath, org.apac * Returns the median instant time between the given two instant time. */ public static Option medianInstantTime(String highVal, String lowVal) { - long high = HoodieActiveTimeline.parseDateFromInstantTimeSafely(highVal) + long high = TimelineUtils.parseDateFromInstantTimeSafely(highVal) .orElseThrow(() -> new HoodieException("Get instant time diff with interval [" + highVal + "] error")).getTime(); - long low = HoodieActiveTimeline.parseDateFromInstantTimeSafely(lowVal) + long low = TimelineUtils.parseDateFromInstantTimeSafely(lowVal) .orElseThrow(() -> new HoodieException("Get instant time diff with interval [" + lowVal + "] error")).getTime(); ValidationUtils.checkArgument(high > low, "Instant [" + highVal + "] should have newer timestamp than instant [" + lowVal + "]"); long median = low + (high - low) / 2; - final String instantTime = HoodieActiveTimeline.formatDate(new Date(median)); - if (HoodieTimeline.compareTimestamps(lowVal, HoodieTimeline.GREATER_THAN_OR_EQUALS, instantTime) - || HoodieTimeline.compareTimestamps(highVal, HoodieTimeline.LESSER_THAN_OR_EQUALS, instantTime)) { + final String instantTime = TimelineUtils.formatDate(new Date(median)); + if (compareTimestamps(lowVal, GREATER_THAN_OR_EQUALS, instantTime) + || compareTimestamps(highVal, LESSER_THAN_OR_EQUALS, instantTime)) { return Option.empty(); } return Option.of(instantTime); @@ -405,9 +407,9 @@ public static Option medianInstantTime(String highVal, String lowVal) { * Returns the time interval in seconds between the given instant time. */ public static long instantTimeDiffSeconds(String newInstantTime, String oldInstantTime) { - long newTimestamp = HoodieActiveTimeline.parseDateFromInstantTimeSafely(newInstantTime) + long newTimestamp = TimelineUtils.parseDateFromInstantTimeSafely(newInstantTime) .orElseThrow(() -> new HoodieException("Get instant time diff with interval [" + oldInstantTime + ", " + newInstantTime + "] error")).getTime(); - long oldTimestamp = HoodieActiveTimeline.parseDateFromInstantTimeSafely(oldInstantTime) + long oldTimestamp = TimelineUtils.parseDateFromInstantTimeSafely(oldInstantTime) .orElseThrow(() -> new HoodieException("Get instant time diff with interval [" + oldInstantTime + ", " + newInstantTime + "] error")).getTime(); return (newTimestamp - oldTimestamp) / 1000; } @@ -455,14 +457,14 @@ public static String getLastPendingInstant(HoodieTableMetaClient metaClient, boo } return metaClient.getCommitsTimeline().filterPendingExcludingCompaction() .lastInstant() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .orElse(null); } public static String getLastCompletedInstant(HoodieTableMetaClient metaClient) { return metaClient.getCommitsTimeline().filterCompletedInstants() .lastInstant() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .orElse(null); } 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 a9bfcb02ba6d..c9a5d4c9ff2f 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 @@ -63,6 +63,7 @@ import java.util.concurrent.CompletableFuture; import java.util.concurrent.TimeUnit; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.CoreMatchers.not; @@ -337,7 +338,7 @@ void testSyncMetadataTable() throws Exception { HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(conf)), new Path(metadataTableBasePath).getParent().getParent().toString()); int metadataPartitions = dataTableMetaClient.getTableConfig().getMetadataPartitions().size(); assertThat("Instants needed to sync to metadata table do not match", completedTimeline.countInstants(), is(metadataPartitions)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // test metadata table compaction // write few more commits until compaction @@ -347,7 +348,7 @@ void testSyncMetadataTable() throws Exception { metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(numCommits + 1)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), is(instant)); } // the 5th commit triggers the compaction mockWriteWithMetadata(); @@ -361,7 +362,7 @@ void testSyncMetadataTable() throws Exception { metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(i + 1)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), is(instant)); } // write another commit to trigger clean @@ -410,7 +411,7 @@ void testSyncMetadataTableWithLogCompaction() throws Exception { HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(conf)), new Path(metadataTableBasePath).getParent().getParent().toString()); int metadataPartitions = dataTableMetaClient.getTableConfig().getMetadataPartitions().size(); assertThat("Instants needed to sync to metadata table do not match", completedTimeline.countInstants(), is(metadataPartitions)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // test metadata table log compaction // already 1 commit is used to initialized FILES partition in MDT @@ -421,7 +422,7 @@ void testSyncMetadataTableWithLogCompaction() throws Exception { metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(numCommits + 1)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), is(instant)); } // the 5th commit triggers the log compaction mockWriteWithMetadata(); @@ -429,7 +430,7 @@ void testSyncMetadataTableWithLogCompaction() throws Exception { completedTimeline = metadataTableMetaClient.reloadActiveTimeline().filterCompletedAndCompactionInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(numCommits + 2)); assertThat("The log compaction instant time should be new generated", - completedTimeline.nthFromLastInstant(1).get().getTimestamp(), not(instant)); + completedTimeline.nthFromLastInstant(1).get().requestedTime(), not(instant)); // log compaction is another delta commit assertThat(completedTimeline.nthFromLastInstant(1).get().getAction(), is(HoodieTimeline.DELTA_COMMIT_ACTION)); } @@ -460,14 +461,14 @@ void testSyncMetadataTableWithRollback() throws Exception { HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(conf)), new Path(metadataTableBasePath).getParent().getParent().toString()); int metadataPartitions = dataTableMetaClient.getTableConfig().getMetadataPartitions().size(); assertThat("Instants needed to sync to metadata table do not match", completedTimeline.countInstants(), is(metadataPartitions)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); // writes a normal commit mockWriteWithMetadata(); instant = coordinator.getInstant(); // creates an inflight commit on the metadata timeline metadataTableMetaClient.getActiveTimeline() - .createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieActiveTimeline.DELTA_COMMIT_ACTION, instant)); + .createNewInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieActiveTimeline.DELTA_COMMIT_ACTION, instant)); metadataTableMetaClient.getActiveTimeline().transitionRequestedToInflight(HoodieActiveTimeline.DELTA_COMMIT_ACTION, instant); metadataTableMetaClient.reloadActiveTimeline(); @@ -477,7 +478,7 @@ void testSyncMetadataTableWithRollback() throws Exception { completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(metadataPartitions + 3)); - assertThat(completedTimeline.nthFromLastInstant(1).get().getTimestamp(), is(instant)); + assertThat(completedTimeline.nthFromLastInstant(1).get().requestedTime(), is(instant)); assertThat("The pending instant should be rolled back first", completedTimeline.lastInstant().get().getAction(), is(HoodieTimeline.ROLLBACK_ACTION)); } @@ -546,13 +547,13 @@ void testLockForMetadataTable() throws Exception { HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(conf)), new Path(metadataTableBasePath).getParent().getParent().toString()); int metadataPartitions = dataTableMetaClient.getTableConfig().getMetadataPartitions().size(); assertThat("Instants needed to sync to metadata table do not match", completedTimeline.countInstants(), is(metadataPartitions)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), startsWith(HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP)); instant = mockWriteWithMetadata(); metadataTableMetaClient.reloadActiveTimeline(); completedTimeline = metadataTableMetaClient.getActiveTimeline().filterCompletedInstants(); assertThat("One instant need to sync to metadata table", completedTimeline.countInstants(), is(metadataPartitions + 1)); - assertThat(completedTimeline.lastInstant().get().getTimestamp(), is(instant)); + assertThat(completedTimeline.lastInstant().get().requestedTime(), is(instant)); } @Test diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java index 96cc3569e830..ef1c6688e72d 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/bucket/ITTestBucketStreamWrite.java @@ -51,6 +51,7 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -100,11 +101,11 @@ private static void doDeleteCommit(String tablePath, boolean isCow) throws Excep // rollback path structure: tablePath/.hoodie/.temp/${commitInstant}/${partition}/${fileGroup}_${fileInstant}.parquet.marker.APPEND HoodieInstant instant = activeCompletedTimeline.getInstants().get(0); - String commitInstant = instant.getTimestamp(); - String filename = activeCompletedTimeline.getInstants().get(0).getFileName(); + String commitInstant = instant.requestedTime(); + String filename = INSTANT_FILE_NAME_GENERATOR.getFileName(activeCompletedTimeline.getInstants().get(0)); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); // delete successful commit to simulate an unsuccessful write diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ClusteringCommitTestSink.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ClusteringCommitTestSink.java index 96f8d7423a11..5f10fd12d77c 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ClusteringCommitTestSink.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ClusteringCommitTestSink.java @@ -39,7 +39,7 @@ public ClusteringCommitTestSink(Configuration conf) { public void invoke(ClusteringCommitEvent event, Context context) throws Exception { super.invoke(event, context); List instants = writeClient.getHoodieTable().getMetaClient().getActiveTimeline().getInstants(); - boolean committed = instants.stream().anyMatch(i -> i.getTimestamp().equals(event.getInstant()) && i.isCompleted()); + boolean committed = instants.stream().anyMatch(i -> i.requestedTime().equals(event.getInstant()) && i.isCompleted()); if (committed && getRuntimeContext().getAttemptNumber() == 0) { throw new HoodieException("Fail first attempt to simulate failover in test."); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java index 2736cdd119cc..f1e72a9ad015 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/cluster/ITTestHoodieFlinkClustering.java @@ -81,6 +81,7 @@ import java.util.concurrent.TimeUnit; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -175,7 +176,7 @@ public void testHoodieFlinkClustering() throws Exception { HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); // Mark instant as clustering inflight - HoodieInstant instant = HoodieTimeline.getClusteringCommitRequestedInstant(clusteringInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getClusteringCommitRequestedInstant(clusteringInstantTime); table.getActiveTimeline().transitionClusterRequestedToInflight(instant, Option.empty()); final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); @@ -384,7 +385,7 @@ public void testHoodieFlinkClusteringScheduleAfterArchive() throws Exception { HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); // Mark instant as clustering inflight - HoodieInstant instant = HoodieTimeline.getClusteringCommitRequestedInstant(firstClusteringInstant); + HoodieInstant instant = INSTANT_GENERATOR.getClusteringCommitRequestedInstant(firstClusteringInstant); table.getActiveTimeline().transitionClusterRequestedToInflight(instant, Option.empty()); final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); @@ -581,7 +582,7 @@ public void testHoodieFlinkClusteringWithTimestampMicros() throws Exception { HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); // Mark instant as clustering inflight - HoodieInstant instant = HoodieTimeline.getClusteringCommitRequestedInstant(clusteringInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getClusteringCommitRequestedInstant(clusteringInstantTime); table.getActiveTimeline().transitionClusterRequestedToInflight(instant, Option.empty()); DataStream dataStream = env.addSource(new ClusteringPlanSourceFunction(clusteringInstantTime, clusteringPlan, conf)) @@ -702,7 +703,7 @@ private void runOfflineCluster(TableEnvironment tableEnv, Configuration conf) th HoodieClusteringPlan clusteringPlan = clusteringPlanOption.get().getRight(); // Mark instant as clustering inflight - HoodieInstant instant = HoodieTimeline.getClusteringCommitRequestedInstant(clusteringInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getClusteringCommitRequestedInstant(clusteringInstantTime); table.getActiveTimeline().transitionClusterRequestedToInflight(instant, Option.empty()); final Schema tableAvroSchema = StreamerUtil.getTableAvroSchema(table.getMetaClient(), false); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java index 4c4b5e5f2dd4..22b1c2d1767a 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java @@ -20,6 +20,7 @@ import org.apache.hudi.client.common.HoodieFlinkEngineContext; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.exception.HoodieException; @@ -40,12 +41,12 @@ public CompactionCommitTestSink(Configuration conf) { public void invoke(CompactionCommitEvent event, Context context) throws Exception { super.invoke(event, context); List instants = writeClient.getHoodieTable().getMetaClient().getActiveTimeline().getInstants(); - boolean compactCommitted = instants.stream().anyMatch(i -> i.getTimestamp().equals(event.getInstant()) && i.isCompleted()); + boolean compactCommitted = instants.stream().anyMatch(i -> i.requestedTime().equals(event.getInstant()) && i.isCompleted()); if (compactCommitted && getRuntimeContext().getAttemptNumber() == 0) { // archive compact instant this.writeClient.getConfig().setValue(HoodieArchivalConfig.MAX_COMMITS_TO_KEEP, "1"); this.writeClient.getConfig().setValue(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP, "1"); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(this.writeClient.getConfig(), this.writeClient.getHoodieTable()); + HoodieTimelineArchiver archiver = new TimelineArchiverV2(this.writeClient.getConfig(), this.writeClient.getHoodieTable()); this.writeClient.getHoodieTable().getMetaClient().reloadActiveTimeline(); archiver.archiveIfRequired(HoodieFlinkEngineContext.DEFAULT); throw new HoodieException("Fail first attempt to simulate failover in test."); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java index baa494c8422d..ab687b25f6fe 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/ITTestHoodieFlinkCompactor.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; @@ -73,6 +72,7 @@ import java.util.Set; import java.util.concurrent.TimeUnit; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -162,7 +162,7 @@ public void testHoodieFlinkCompactor(boolean enableChangelog) throws Exception { HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( table.getMetaClient(), compactionInstantTime); - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime); // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); @@ -236,12 +236,13 @@ public void testHoodieFlinkCompactorWithUpgradeAndDowngrade(boolean upgrade) thr new UpgradeDowngrade(metaClient, writeClient.getConfig(), writeClient.getEngineContext(), FlinkUpgradeDowngradeHelper.getInstance()).run(HoodieTableVersion.SIX, "none"); } + table = writeClient.getHoodieTable(); // generate compaction plan // should support configurable commit metadata HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( table.getMetaClient(), compactionInstantTime); - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime); // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); @@ -362,7 +363,7 @@ public void testHoodieFlinkCompactorWithPlanSelectStrategy(boolean enableChangel // Mark instant as compaction inflight for (String compactionInstantTime : compactionInstantTimeList) { - HoodieInstant hoodieInstant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant hoodieInstant = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime); table.getActiveTimeline().transitionCompactionRequestedToInflight(hoodieInstant); } table.getMetaClient().reloadActiveTimeline(); @@ -483,7 +484,7 @@ private void runOfflineCompact(TableEnvironment tableEnv, Configuration conf) th HoodieCompactionPlan compactionPlan = CompactionUtils.getCompactionPlan( table.getMetaClient(), compactionInstantTime); - HoodieInstant instant = HoodieTimeline.getCompactionRequestedInstant(compactionInstantTime); + HoodieInstant instant = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstantTime); // Mark instant as compaction inflight table.getActiveTimeline().transitionCompactionRequestedToInflight(instant); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java index edce1744cfa1..6108d8560627 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/TestCompactionPlanStrategy.java @@ -18,9 +18,9 @@ package org.apache.hudi.sink.compact; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategies; import org.apache.hudi.sink.compact.strategy.CompactionPlanStrategy; @@ -30,6 +30,7 @@ import java.util.Arrays; import java.util.List; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; /** @@ -40,12 +41,12 @@ public class TestCompactionPlanStrategy { private HoodieTimeline emptyTimeline; private HoodieTimeline allCompleteTimeline; - private static final HoodieInstant INSTANT_001 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); - private static final HoodieInstant INSTANT_002 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "002"); - private static final HoodieInstant INSTANT_003 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"); - private static final HoodieInstant INSTANT_004 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "004"); - private static final HoodieInstant INSTANT_005 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, "005"); - private static final HoodieInstant INSTANT_006 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"); + private static final HoodieInstant INSTANT_001 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); + private static final HoodieInstant INSTANT_002 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "002"); + private static final HoodieInstant INSTANT_003 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "003"); + private static final HoodieInstant INSTANT_004 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "004"); + private static final HoodieInstant INSTANT_005 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMPACTION_ACTION, "005"); + private static final HoodieInstant INSTANT_006 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "006"); @BeforeEach public void beforeEach() { @@ -139,10 +140,10 @@ private void assertHoodieInstantsEquals(HoodieInstant[] expected, List lastCompletedInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant(); - HoodieActiveTimeline.deleteInstantFile( - metaClient.getStorage(), metaClient.getMetaPath(), lastCompletedInstant.get()); + TimelineUtils.deleteInstantFile( + metaClient.getStorage(), metaClient.getMetaPath(), lastCompletedInstant.get(), + metaClient.getInstantFileNameGenerator()); // refresh the heartbeat in case it is timed out. OutputStream outputStream = metaClient.getStorage().create(new StoragePath( HoodieTableMetaClient.getHeartbeatFolderPath(basePath) diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java index 20fe5b1fe951..02ea8ad553f4 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestIncrementalInputSplits.java @@ -69,9 +69,11 @@ import java.util.stream.IntStream; import java.util.stream.Stream; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertIterableEquals; @@ -95,16 +97,16 @@ void testFilterInstantsWithRange() throws IOException { metaClient = HoodieTestUtils.init(basePath, HoodieTableType.MERGE_ON_READ); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - HoodieInstant commit1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); - HoodieInstant commit2 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); - HoodieInstant commit3 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "3"); + HoodieInstant commit1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); + HoodieInstant commit2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); + HoodieInstant commit3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "3"); timeline.createCompleteInstant(commit1); timeline.createCompleteInstant(commit2); timeline.createCompleteInstant(commit3); timeline = metaClient.reloadActiveTimeline(); Map completionTimeMap = timeline.filterCompletedInstants().getInstantsAsStream() - .collect(Collectors.toMap(HoodieInstant::getTimestamp, HoodieInstant::getCompletionTime)); + .collect(Collectors.toMap(HoodieInstant::requestedTime, HoodieInstant::getCompletionTime)); IncrementalQueryAnalyzer analyzer1 = IncrementalQueryAnalyzer.builder() .metaClient(metaClient) @@ -140,7 +142,7 @@ void testFilterInstantsWithRange() throws IOException { assertIterableEquals(Arrays.asList(commit1, commit2, commit3), activeInstants3); // add an inflight instant which should be excluded - HoodieInstant commit4 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "4"); + HoodieInstant commit4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "4"); timeline.createNewInstant(commit4); timeline = metaClient.reloadActiveTimeline(); assertEquals(4, timeline.getInstants().size()); @@ -154,13 +156,13 @@ void testFilterInstantsByConditionForMOR() throws IOException { HoodieActiveTimeline timelineMOR = metaClient.getActiveTimeline(); // commit1: delta commit - HoodieInstant commit1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); + HoodieInstant commit1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "1"); timelineMOR.createCompleteInstant(commit1); // commit2: delta commit - HoodieInstant commit2 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); + HoodieInstant commit2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); // commit3: clustering timelineMOR.createCompleteInstant(commit2); - HoodieInstant commit3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, "3"); + HoodieInstant commit3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, "3"); timelineMOR.createNewInstant(commit3); commit3 = timelineMOR.transitionClusterRequestedToInflight(commit3, Option.empty()); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata( @@ -170,11 +172,10 @@ void testFilterInstantsByConditionForMOR() throws IOException { WriteOperationType.CLUSTER, "", HoodieTimeline.REPLACE_COMMIT_ACTION); - timelineMOR.transitionClusterInflightToComplete(true, - HoodieTimeline.getClusteringCommitInflightInstant(commit3.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + timelineMOR.transitionClusterInflightToComplete(true, INSTANT_GENERATOR.getClusteringCommitInflightInstant(commit3.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // commit4: insert overwrite - HoodieInstant commit4 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4"); + HoodieInstant commit4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4"); timelineMOR.createNewInstant(commit4); commit4 = timelineMOR.transitionReplaceRequestedToInflight(commit4, Option.empty()); commitMetadata = CommitUtils.buildMetadata( @@ -184,11 +185,10 @@ void testFilterInstantsByConditionForMOR() throws IOException { WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); - timelineMOR.transitionReplaceInflightToComplete(true, - HoodieTimeline.getReplaceCommitInflightInstant(commit4.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + timelineMOR.transitionReplaceInflightToComplete(true, INSTANT_GENERATOR.getReplaceCommitInflightInstant(commit4.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // commit5: insert overwrite table - HoodieInstant commit5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "5"); + HoodieInstant commit5 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "5"); timelineMOR.createNewInstant(commit5); commit5 = timelineMOR.transitionReplaceRequestedToInflight(commit5, Option.empty()); commitMetadata = CommitUtils.buildMetadata( @@ -198,11 +198,10 @@ void testFilterInstantsByConditionForMOR() throws IOException { WriteOperationType.INSERT_OVERWRITE_TABLE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); - timelineMOR.transitionReplaceInflightToComplete(true, - HoodieTimeline.getReplaceCommitInflightInstant(commit5.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + timelineMOR.transitionReplaceInflightToComplete(true, INSTANT_GENERATOR.getReplaceCommitInflightInstant(commit5.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // commit6: compaction - HoodieInstant commit6 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "6"); + HoodieInstant commit6 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "6"); timelineMOR.createNewInstant(commit6); commit6 = timelineMOR.transitionCompactionRequestedToInflight(commit6); commit6 = timelineMOR.transitionCompactionInflightToComplete(false, commit6, Option.empty()); @@ -235,13 +234,13 @@ void testFilterInstantsByConditionForCOW() throws IOException { HoodieActiveTimeline timelineCOW = metaClient.getActiveTimeline(); // commit1: commit - HoodieInstant commit1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant commit1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); timelineCOW.createCompleteInstant(commit1); // commit2: commit - HoodieInstant commit2 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant commit2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); // commit3: clustering timelineCOW.createCompleteInstant(commit2); - HoodieInstant commit3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, "3"); + HoodieInstant commit3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, "3"); timelineCOW.createNewInstant(commit3); commit3 = timelineCOW.transitionClusterRequestedToInflight(commit3, Option.empty()); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata( @@ -252,10 +251,10 @@ void testFilterInstantsByConditionForCOW() throws IOException { "", HoodieTimeline.REPLACE_COMMIT_ACTION); timelineCOW.transitionClusterInflightToComplete(true, - HoodieTimeline.getClusteringCommitInflightInstant(commit3.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + INSTANT_GENERATOR.getClusteringCommitInflightInstant(commit3.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // commit4: insert overwrite - HoodieInstant commit4 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4"); + HoodieInstant commit4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4"); timelineCOW.createNewInstant(commit4); commit4 = timelineCOW.transitionReplaceRequestedToInflight(commit4, Option.empty()); commitMetadata = CommitUtils.buildMetadata( @@ -265,11 +264,10 @@ void testFilterInstantsByConditionForCOW() throws IOException { WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); - timelineCOW.transitionReplaceInflightToComplete(true, - HoodieTimeline.getReplaceCommitInflightInstant(commit4.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + timelineCOW.transitionReplaceInflightToComplete(true, INSTANT_GENERATOR.getReplaceCommitInflightInstant(commit4.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // commit5: insert overwrite table - HoodieInstant commit5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "5"); + HoodieInstant commit5 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "5"); timelineCOW.createNewInstant(commit5); commit5 = timelineCOW.transitionReplaceRequestedToInflight(commit5, Option.empty()); commitMetadata = CommitUtils.buildMetadata( @@ -279,9 +277,8 @@ void testFilterInstantsByConditionForCOW() throws IOException { WriteOperationType.INSERT_OVERWRITE_TABLE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); - timelineCOW.transitionReplaceInflightToComplete(true, - HoodieTimeline.getReplaceCommitInflightInstant(commit5.getTimestamp()), - serializeCommitMetadata(commitMetadata)); + timelineCOW.transitionReplaceInflightToComplete(true, INSTANT_GENERATOR.getReplaceCommitInflightInstant(commit5.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); timelineCOW = timelineCOW.reload(); @@ -432,11 +429,11 @@ void testInputSplitsWithSpeedLimit() throws Exception { String minStartCommit = result.getInputSplits().stream() .map(split -> split.getInstantRange().get().getStartInstant().get()) - .min((commit1,commit2) -> HoodieTimeline.compareTimestamps(commit1, LESSER_THAN, commit2) ? 1 : 0) + .min((commit1,commit2) -> compareTimestamps(commit1, LESSER_THAN, commit2) ? 1 : 0) .orElse(null); String maxEndCommit = result.getInputSplits().stream() .map(split -> split.getInstantRange().get().getEndInstant().get()) - .max((commit1,commit2) -> HoodieTimeline.compareTimestamps(commit1, GREATER_THAN, commit2) ? 1 : 0) + .max((commit1,commit2) -> compareTimestamps(commit1, GREATER_THAN, commit2) ? 1 : 0) .orElse(null); assertEquals(0, intervalBetween2Instants(commitsTimeline, minStartCommit, maxEndCommit), "Should read 1 instant"); } @@ -460,7 +457,7 @@ void testInputSplitsForSplitLastCommit() throws Exception { HoodieTimeline commitsTimeline = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants(); List instants = commitsTimeline.getInstants(); - String lastInstant = commitsTimeline.lastInstant().map(HoodieInstant::getTimestamp).get(); + String lastInstant = commitsTimeline.lastInstant().map(HoodieInstant::requestedTime).get(); List metadataList = instants.stream() .map(instant -> WriteProfiles.getCommitMetadata(tableName, new Path(basePath), instant, commitsTimeline)).collect(Collectors.toList()); @@ -558,7 +555,7 @@ private Integer intervalBetween2Instants(HoodieTimeline timeline, String instant private Integer getInstantIdxInTimeline(HoodieTimeline timeline, String instant) { List instants = timeline.getInstants(); return IntStream.range(0, instants.size()) - .filter(i -> instants.get(i).getTimestamp().equals(instant)) + .filter(i -> instants.get(i).requestedTime().equals(instant)) .findFirst() .orElse(-1); } diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java index db28ce1326f3..a0308cfa8a9c 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/source/TestStreamReadMonitoringFunction.java @@ -172,7 +172,7 @@ public void testConsumeForSpeedLimitWhenEmptyCommitExists() throws Exception { conf.set(FlinkOptions.READ_STREAMING_SKIP_CLUSTERING, true); conf.set(FlinkOptions.READ_STREAMING_SKIP_COMPACT, true); conf.set(FlinkOptions.READ_COMMITS_LIMIT, 2); - conf.set(FlinkOptions.READ_START_COMMIT, String.valueOf((Long.valueOf(firstInstant.getTimestamp()) - 100))); + conf.set(FlinkOptions.READ_START_COMMIT, String.valueOf((Long.valueOf(firstInstant.requestedTime()) - 100))); StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf); try (AbstractStreamOperatorTestHarness harness = createHarness(function)) { harness.setup(); @@ -280,9 +280,9 @@ public void testConsumingHollowInstants() throws Exception { List instants = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants(); assertThat(instants.size(), is(2)); - String c2 = oriInstants.get(1).getTimestamp(); - String c3 = oriInstants.get(2).getTimestamp(); - String c4 = instants.get(1).getTimestamp(); + String c2 = oriInstants.get(1).requestedTime(); + String c3 = oriInstants.get(2).requestedTime(); + String c4 = instants.get(1).requestedTime(); conf.setString(FlinkOptions.READ_START_COMMIT, FlinkOptions.START_COMMIT_EARLIEST); StreamReadMonitoringFunction function = TestUtils.getMonitorFunc(conf); 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 ee43902ddd77..6f9d9ba18595 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 @@ -77,6 +77,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.hamcrest.CoreMatchers.instanceOf; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -579,9 +580,9 @@ void testReadHollowInstants(HoodieTableType tableType) throws Exception { List instants = metaClient.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants(); assertThat(instants.size(), is(2)); - String c2 = oriInstants.get(1).getTimestamp(); - String c3 = oriInstants.get(2).getTimestamp(); - String c4 = oriInstants.get(3).getTimestamp(); + String c2 = oriInstants.get(1).requestedTime(); + String c3 = oriInstants.get(2).requestedTime(); + String c4 = oriInstants.get(3).requestedTime(); InputFormat inputFormat = this.tableSource.getInputFormat(true); assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class)); @@ -1204,8 +1205,8 @@ void testIncReadWithNonBlockingConcurrencyControl(boolean skipCompaction) throws assertTrue(firstCommit.isPresent()); assertThat(firstCommit.get().getAction(), is(HoodieTimeline.DELTA_COMMIT_ACTION)); - java.nio.file.Path metaFilePath = Paths.get(metaClient.getMetaPath().toString(), firstCommit.get().getFileName()); - String newCompletionTime = TestUtils.amendCompletionTimeToLatest(metaClient, metaFilePath, firstCommit.get().getTimestamp()); + java.nio.file.Path metaFilePath = Paths.get(metaClient.getMetaPath().toString(), INSTANT_FILE_NAME_GENERATOR.getFileName(firstCommit.get())); + String newCompletionTime = TestUtils.amendCompletionTimeToLatest(metaClient, metaFilePath, firstCommit.get().requestedTime()); InputFormat inputFormat = this.tableSource.getInputFormat(true); assertThat(inputFormat, instanceOf(MergeOnReadInputFormat.class)); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestClusteringUtil.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestClusteringUtil.java index 38ae95a9d2be..f35a2792731f 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestClusteringUtil.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestClusteringUtil.java @@ -52,6 +52,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -110,7 +111,7 @@ void rollbackClustering() throws Exception { .stream().allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED); assertTrue(allRolledBack, "all the instants should be rolled back"); List actualInstants = ClusteringUtils.getPendingClusteringInstantTimes(table.getMetaClient()) - .stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .stream().map(HoodieInstant::requestedTime).collect(Collectors.toList()); assertThat(actualInstants, is(oriInstants)); } @@ -135,7 +136,7 @@ private String generateClusteringPlan() { plan, Collections.emptyMap(), 1); String instantTime = table.getMetaClient().createNewInstantTime(); HoodieInstant clusteringInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, instantTime); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, instantTime); try { metaClient.getActiveTimeline().saveToPendingClusterCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(metadata)); 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 3845b6150e51..111680bdcdbf 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 @@ -51,6 +51,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -102,7 +103,7 @@ void rollbackCompaction() throws Exception { .allMatch(instant -> instant.getState() == HoodieInstant.State.REQUESTED); assertTrue(allRolledBack, "all the instants should be rolled back"); List actualInstants = metaClient.getActiveTimeline() - .filterPendingCompactionTimeline().getInstantsAsStream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .filterPendingCompactionTimeline().getInstantsAsStream().map(HoodieInstant::requestedTime).collect(Collectors.toList()); assertThat(actualInstants, is(oriInstants)); } @@ -124,7 +125,7 @@ void rollbackEarliestCompaction() throws Exception { String instantTime = metaClient.getActiveTimeline() .filterPendingCompactionTimeline().filter(instant -> instant.getState() == HoodieInstant.State.REQUESTED) - .firstInstant().get().getTimestamp(); + .firstInstant().get().requestedTime(); assertThat(instantTime, is(oriInstants.get(0))); } @@ -176,7 +177,7 @@ private String generateCompactionPlan() { HoodieCompactionPlan plan = new HoodieCompactionPlan(Collections.singletonList(operation), Collections.emptyMap(), 1, null, null); String instantTime = table.getMetaClient().createNewInstantTime(); HoodieInstant compactionInstant = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); try { metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java index 3dd5d1830497..b9745adf80c0 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestData.java @@ -887,7 +887,7 @@ public static void checkWrittenDataMOR( Schema schema = new TableSchemaResolver(metaClient).getTableAvroSchema(); String latestInstant = metaClient.getActiveTimeline().filterCompletedInstants() - .lastInstant().map(HoodieInstant::getTimestamp).orElse(null); + .lastInstant().map(HoodieInstant::requestedTime).orElse(null); assertNotNull(latestInstant, "No completed commit under table path" + basePath); File[] partitionDirs = baseFile.listFiles(file -> !file.getName().startsWith(".") && file.isDirectory()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java index e5aa9ed405ea..bc02f8a37911 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/utils/TestUtils.java @@ -20,7 +20,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.HoodieTableVersion; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineUtils; @@ -44,6 +44,7 @@ import java.nio.file.Files; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertTrue; /** @@ -68,7 +69,7 @@ public static String getLastCompleteInstant(String basePath, String commitAction return metaClient.getCommitsTimeline().filterCompletedInstants() .filter(instant -> commitAction.equals(instant.getAction())) .lastInstant() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .orElse(null); } @@ -78,7 +79,7 @@ public static String getLastDeltaCompleteInstant(String basePath) { return metaClient.getCommitsTimeline().filterCompletedInstants() .filter(hoodieInstant -> hoodieInstant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION)) .lastInstant() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .orElse(null); } @@ -86,17 +87,17 @@ public static String getFirstCompleteInstant(String basePath) { final HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient( new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new Configuration())), basePath); return metaClient.getCommitsAndCompactionTimeline().filterCompletedInstants().firstInstant() - .map(HoodieInstant::getTimestamp).orElse(null); + .map(HoodieInstant::requestedTime).orElse(null); } @Nullable public static String getNthCompleteInstant(StoragePath basePath, int n, String action) { final HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient( - new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new Configuration())), basePath); + new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new Configuration())), basePath, HoodieTableVersion.EIGHT); return metaClient.getActiveTimeline() .filterCompletedInstants() .filter(instant -> action.equals(instant.getAction())) - .nthInstant(n).map(HoodieInstant::getTimestamp) + .nthInstant(n).map(HoodieInstant::requestedTime) .orElse(null); } @@ -105,7 +106,7 @@ public static String getNthArchivedInstant(String basePath, int n) { final HoodieTableMetaClient metaClient = HoodieTestUtils.createMetaClient( new HadoopStorageConfiguration(HadoopConfigurations.getHadoopConf(new Configuration())), basePath); return metaClient.getArchivedTimeline().getCommitsTimeline().filterCompletedInstants() - .nthInstant(n).map(HoodieInstant::getTimestamp).orElse(null); + .nthInstant(n).map(HoodieInstant::requestedTime).orElse(null); } public static String getSplitPartitionPath(MergeOnReadInputSplit split) { @@ -136,14 +137,14 @@ public static int getCompletedInstantCount(String basePath, String action) { public static HoodieCommitMetadata deleteInstantFile(HoodieTableMetaClient metaClient, HoodieInstant instant) throws Exception { ValidationUtils.checkArgument(instant.isCompleted()); HoodieCommitMetadata metadata = TimelineUtils.getCommitMetadata(instant, metaClient.getActiveTimeline()); - HoodieActiveTimeline.deleteInstantFile(metaClient.getStorage(), metaClient.getMetaPath(), - instant); + TimelineUtils.deleteInstantFile(metaClient.getStorage(), metaClient.getMetaPath(), + instant, metaClient.getInstantFileNameGenerator()); return metadata; } public static void saveInstantAsComplete(HoodieTableMetaClient metaClient, HoodieInstant instant, HoodieCommitMetadata metadata) throws Exception { - metaClient.getActiveTimeline().saveAsComplete(new HoodieInstant(true, instant.getAction(), instant.getTimestamp()), - serializeCommitMetadata(metadata)); + metaClient.getActiveTimeline().saveAsComplete(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instant.getAction(), instant.requestedTime()), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata)); } public static String amendCompletionTimeToLatest(HoodieTableMetaClient metaClient, java.nio.file.Path sourcePath, String instantTime) throws IOException { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java index ad5b1230bde6..bd7f7d167196 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/fs/TestFSUtils.java @@ -22,7 +22,7 @@ import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.cdc.HoodieCDCUtils; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.CollectionUtils; @@ -91,7 +91,7 @@ public void tearDown() throws Exception { @Test public void testMakeDataFileName() { - String instantTime = HoodieActiveTimeline.formatDate(new Date()); + String instantTime = TimelineUtils.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); assertEquals(FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName, HoodieCommonTestHarness.BASE_FILE_EXTENSION), fileName + "_" + TEST_WRITE_TOKEN + "_" + instantTime + HoodieCommonTestHarness.BASE_FILE_EXTENSION); @@ -99,7 +99,7 @@ public void testMakeDataFileName() { @Test public void testMaskFileName() { - String instantTime = HoodieActiveTimeline.formatDate(new Date()); + String instantTime = TimelineUtils.formatDate(new Date()); int taskPartitionId = 2; assertEquals(FSUtils.maskWithoutFileId(instantTime, taskPartitionId), "*_" + taskPartitionId + "_" + instantTime + HoodieCommonTestHarness.BASE_FILE_EXTENSION); } @@ -167,7 +167,7 @@ public void testProcessFiles() throws Exception { @Test public void testGetCommitTime() { - String instantTime = HoodieActiveTimeline.formatDate(new Date()); + String instantTime = TimelineUtils.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName, HoodieCommonTestHarness.BASE_FILE_EXTENSION); assertEquals(instantTime, FSUtils.getCommitTime(fullFileName)); @@ -178,7 +178,7 @@ public void testGetCommitTime() { @Test public void testGetFileNameWithoutMeta() { - String instantTime = HoodieActiveTimeline.formatDate(new Date()); + String instantTime = TimelineUtils.formatDate(new Date()); String fileName = UUID.randomUUID().toString(); String fullFileName = FSUtils.makeBaseFileName(instantTime, TEST_WRITE_TOKEN, fileName, HoodieCommonTestHarness.BASE_FILE_EXTENSION); assertEquals(fileName, FSUtils.getFileId(fullFileName)); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/model/TestHoodieFileGroup.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/model/TestHoodieFileGroup.java index 0749c710df0e..2fbdcdc1a8e4 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/model/TestHoodieFileGroup.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/model/TestHoodieFileGroup.java @@ -18,7 +18,7 @@ package org.apache.hudi.common.model; -import org.apache.hudi.common.table.timeline.CompletionTimeQueryView; +import org.apache.hudi.common.table.timeline.versioning.v2.CompletionTimeQueryViewV2; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.MockHoodieTimeline; @@ -33,6 +33,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -68,11 +69,11 @@ public void testCommittedFileSlices() { @Test public void testCommittedFileSlicesWithSavepointAndHoles() { MockHoodieTimeline activeTimeline = new MockHoodieTimeline(Stream.of( - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well ).collect(Collectors.toList())); HoodieFileGroup fileGroup = new HoodieFileGroup("", "data", activeTimeline.filterCompletedAndCompactionInstants()); for (int i = 0; i < 7; i++) { @@ -89,14 +90,14 @@ public void testCommittedFileSlicesWithSavepointAndHoles() { @Test public void testGetBaseInstantTime() { MockHoodieTimeline activeTimeline = new MockHoodieTimeline(Stream.of( - new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001", "001"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002", "011"), // finishes in the last - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003", "007"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "004", "006"), - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "005", "007") + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001", "001"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "002", "011"), // finishes in the last + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "003", "007"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "004", "006"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "005", "007") ).collect(Collectors.toList())); - CompletionTimeQueryView queryView = getMockCompletionTimeQueryView(activeTimeline); + CompletionTimeQueryViewV2 queryView = getMockCompletionTimeQueryView(activeTimeline); HoodieFileGroup fileGroup = new HoodieFileGroup("", "data", activeTimeline.filterCompletedAndCompactionInstants()); @@ -133,10 +134,10 @@ public void testGetBaseInstantTime() { fileGroup.getBaseInstantTime(queryView, logFile2), is("005")); } - private CompletionTimeQueryView getMockCompletionTimeQueryView(MockHoodieTimeline activeTimeline) { + private CompletionTimeQueryViewV2 getMockCompletionTimeQueryView(MockHoodieTimeline activeTimeline) { Map completionTimeMap = activeTimeline.filterCompletedInstants().getInstantsAsStream() - .collect(Collectors.toMap(HoodieInstant::getTimestamp, HoodieInstant::getCompletionTime)); - CompletionTimeQueryView queryView = mock(CompletionTimeQueryView.class); + .collect(Collectors.toMap(HoodieInstant::requestedTime, HoodieInstant::getCompletionTime)); + CompletionTimeQueryViewV2 queryView = mock(CompletionTimeQueryViewV2.class); when(queryView.getCompletionTime(any(String.class), any(String.class))) .thenAnswer((InvocationOnMock invocationOnMock) -> { String instantTime = invocationOnMock.getArgument(1); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java index 66f981155155..dcc5d3001e18 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestHoodieTableMetaClient.java @@ -35,6 +35,7 @@ import java.io.IOException; import java.util.Properties; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -78,13 +79,13 @@ public void testSerDe() { HoodieTestUtils.serializeDeserialize(metaClient, HoodieTableMetaClient.class); assertNotNull(deserializedMetaClient); HoodieActiveTimeline commitTimeline = deserializedMetaClient.getActiveTimeline(); - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "1"); commitTimeline.createNewInstant(instant); commitTimeline.saveAsComplete(instant, Option.of(getUTF8Bytes("test-detail"))); commitTimeline = commitTimeline.reload(); HoodieInstant completedInstant = commitTimeline.getInstantsAsStream().findFirst().get(); assertTrue(completedInstant.isCompleted()); - assertEquals(completedInstant.getTimestamp(), instant.getTimestamp()); + assertEquals(completedInstant.requestedTime(), instant.requestedTime()); assertArrayEquals(getUTF8Bytes("test-detail"), commitTimeline.getInstantDetails(completedInstant).get(), "Commit value should be \"test-detail\""); } @@ -95,7 +96,7 @@ public void testCommitTimeline() { HoodieTimeline activeCommitTimeline = activeTimeline.getCommitAndReplaceTimeline(); assertTrue(activeCommitTimeline.empty(), "Should be empty commit timeline"); - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "1"); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getUTF8Bytes("test-detail"))); @@ -109,7 +110,7 @@ public void testCommitTimeline() { activeCommitTimeline = activeTimeline.getCommitAndReplaceTimeline(); assertFalse(activeCommitTimeline.empty(), "Should be the 1 commit we made"); assertTrue(completedInstant.isCompleted()); - assertTrue(completedInstant.getTimestamp().equals(instant.getTimestamp())); + assertTrue(completedInstant.requestedTime().equals(instant.requestedTime())); assertArrayEquals(getUTF8Bytes("test-detail"), activeCommitTimeline.getInstantDetails(completedInstant).get(), "Commit value should be \"test-detail\""); } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java index f82f627323cf..8b2872ca05de 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java @@ -29,14 +29,16 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.BaseTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantComparatorV2; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.MockHoodieTimeline; @@ -77,6 +79,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.SAVEPOINT_ACTION; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.apache.hudi.common.table.timeline.TimelineUtils.handleHollowCommitIfNeeded; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -116,7 +119,8 @@ public void testGetPartitionsWithReplaceOrClusterCommits(boolean withReplace) th String ts1 = "1"; String replacePartition = "2021/01/01"; String newFilePartition = "2021/01/02"; - HoodieInstant instant1 = new HoodieInstant(true, withReplace ? HoodieTimeline.REPLACE_COMMIT_ACTION : HoodieTimeline.CLUSTERING_ACTION, ts1); + HoodieInstant instant1 = new HoodieInstant(INFLIGHT, withReplace ? HoodieTimeline.REPLACE_COMMIT_ACTION : HoodieTimeline.CLUSTERING_ACTION, ts1, + InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant1); // create replace metadata only with replaced file Ids (no new files created) if (withReplace) { @@ -135,7 +139,8 @@ public void testGetPartitionsWithReplaceOrClusterCommits(boolean withReplace) th assertEquals(replacePartition, partitions.get(0)); String ts2 = "2"; - HoodieInstant instant2 = new HoodieInstant(true, withReplace ? HoodieTimeline.REPLACE_COMMIT_ACTION : HoodieTimeline.CLUSTERING_ACTION, ts2); + HoodieInstant instant2 = new HoodieInstant(INFLIGHT, withReplace ? HoodieTimeline.REPLACE_COMMIT_ACTION : HoodieTimeline.CLUSTERING_ACTION, ts2, + InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant2); // create replace metadata only with replaced file Ids (no new files created) if (withReplace) { @@ -167,11 +172,11 @@ public void testGetPartitions() throws IOException { String olderPartition = "0"; // older partitions that is modified by all cleans for (int i = 1; i <= 5; i++) { String ts = i + ""; - HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(INFLIGHT, COMMIT_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, ts, ts, 2, Collections.emptyMap()))); - HoodieInstant cleanInstant = new HoodieInstant(true, CLEAN_ACTION, ts); + HoodieInstant cleanInstant = INSTANT_GENERATOR.createNewInstant(INFLIGHT, CLEAN_ACTION, ts); activeTimeline.createNewInstant(cleanInstant); activeTimeline.saveAsComplete(cleanInstant, getCleanMetadata(olderPartition, ts, false)); } @@ -206,11 +211,11 @@ public void testGetPartitionsUnPartitioned() throws IOException { String partitionPath = ""; for (int i = 1; i <= 5; i++) { String ts = i + ""; - HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(INFLIGHT, COMMIT_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, partitionPath, ts, 2, Collections.emptyMap()))); - HoodieInstant cleanInstant = new HoodieInstant(true, CLEAN_ACTION, ts); + HoodieInstant cleanInstant = new HoodieInstant(INFLIGHT, CLEAN_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(cleanInstant); activeTimeline.saveAsComplete(cleanInstant, getCleanMetadata(partitionPath, ts, false)); } @@ -233,7 +238,7 @@ public void testRestoreInstants() throws Exception { for (int i = 1; i <= 5; i++) { String ts = i + ""; - HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, ts); + HoodieInstant instant = new HoodieInstant(INFLIGHT, HoodieTimeline.RESTORE_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, COMMIT_ACTION))); } @@ -258,12 +263,12 @@ public void testGetExtraMetadata() throws Exception { assertFalse(TimelineUtils.getExtraMetadataFromLatest(metaClient, extraMetadataKey).isPresent()); String ts = "0"; - HoodieInstant instant = new HoodieInstant(true, COMMIT_ACTION, ts); + HoodieInstant instant = new HoodieInstant(INFLIGHT, COMMIT_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant); activeTimeline.saveAsComplete(instant, Option.of(getCommitMetadata(basePath, ts, ts, 2, Collections.emptyMap()))); ts = "1"; - instant = new HoodieInstant(true, COMMIT_ACTION, ts); + instant = new HoodieInstant(INFLIGHT, COMMIT_ACTION, ts, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant); Map extraMetadata = new HashMap<>(); extraMetadata.put(extraMetadataKey, extraMetadataValue1); @@ -277,7 +282,7 @@ public void testGetExtraMetadata() throws Exception { // verify adding clustering commit doesn't change behavior of getExtraMetadataFromLatest String ts2 = "2"; - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, ts2); + HoodieInstant instant2 = new HoodieInstant(INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, ts2, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant2); String newValueForMetadata = "newValue2"; extraMetadata.put(extraMetadataKey, newValueForMetadata); @@ -305,21 +310,21 @@ public void testGetCommitsTimelineAfter() throws IOException { String startTs = "010"; HoodieTableMetaClient mockMetaClient = prepareMetaClient( Arrays.asList( - new HoodieInstant(COMPLETED, COMMIT_ACTION, "009", "013"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012")), - Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002")), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "009", "013", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), + Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), startTs ); // Commit 009 will be included in result because it has greater commit completion than 010 verifyTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, COMMIT_ACTION, "009"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012")), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "009", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getCommitsTimelineAfter(mockMetaClient, startTs, Option.of(startTs))); verify(mockMetaClient, never()).getArchivedTimeline(any()); @@ -327,20 +332,20 @@ public void testGetCommitsTimelineAfter() throws IOException { startTs = "001"; mockMetaClient = prepareMetaClient( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "009", "009"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012")), - Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002")), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "009", "009", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), + Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), startTs ); verifyTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012")), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getCommitsTimelineAfter(mockMetaClient, startTs, Option.of(startTs))); verify(mockMetaClient, times(1)).getArchivedTimeline(any()); @@ -348,26 +353,26 @@ public void testGetCommitsTimelineAfter() throws IOException { startTs = "005"; mockMetaClient = prepareMetaClient( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "003", "003"), - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "007", "007"), - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "009", "009"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012")), - Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "005", "005"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "006", "006"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "008", "008")), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "003", "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "007", "007", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "009", "009", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), + Arrays.asList(new HoodieInstant(COMPLETED, COMMIT_ACTION, "001", "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "002", "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "005", "005", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "006", "006", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "008", "008", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), startTs ); verifyTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, COMMIT_ACTION, "006", "006"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "008", "008"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012")), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "006", "006", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "008", "008", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "012", "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getCommitsTimelineAfter(mockMetaClient, startTs, Option.of(startTs))); verify(mockMetaClient, times(1)).getArchivedTimeline(any()); } @@ -381,23 +386,23 @@ private HoodieTableMetaClient prepareMetaClient( HoodieArchivedTimeline mockArchivedTimeline = mock(HoodieArchivedTimeline.class); when(mockMetaClient.scanHoodieInstantsFromFileSystem(any(), eq(true))) .thenReturn(activeInstants); - HoodieActiveTimeline activeTimeline = new HoodieActiveTimeline(mockMetaClient); + HoodieActiveTimeline activeTimeline = new ActiveTimelineV2(mockMetaClient); when(mockMetaClient.getActiveTimeline()) .thenReturn(activeTimeline); Set validWriteActions = CollectionUtils.createSet( COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); when(mockMetaClient.getArchivedTimeline(any())) .thenReturn(mockArchivedTimeline); - HoodieDefaultTimeline mergedTimeline = new HoodieDefaultTimeline( + HoodieTimeline mergedTimeline = new BaseTimelineV2( archivedInstants.stream() - .filter(instant -> instant.getTimestamp().compareTo(startTs) >= 0), + .filter(instant -> instant.requestedTime().compareTo(startTs) >= 0), i -> Option.empty()) .mergeTimeline(activeTimeline); when(mockArchivedTimeline.mergeTimeline(eq(activeTimeline))) .thenReturn(mergedTimeline); - HoodieDefaultTimeline mergedWriteTimeline = new HoodieDefaultTimeline( + HoodieTimeline mergedWriteTimeline = new BaseTimelineV2( archivedInstants.stream() - .filter(instant -> instant.getTimestamp().compareTo(startTs) >= 0), + .filter(instant -> instant.requestedTime().compareTo(startTs) >= 0), i -> Option.empty()) .mergeTimeline(activeTimeline.getWriteTimeline()); when(mockArchivedTimeline.mergeTimeline(argThat(timeline -> timeline.filter( @@ -424,62 +429,62 @@ public void testGetEarliestInstantForMetadataArchival() throws IOException { // Earlier request clean action before commits assertEquals( - Option.of(new HoodieInstant(REQUESTED, CLEAN_ACTION, "003")), + Option.of(new HoodieInstant(REQUESTED, CLEAN_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getEarliestInstantForMetadataArchival( prepareActiveTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), - new HoodieInstant(COMPLETED, CLEAN_ACTION, "002"), - new HoodieInstant(REQUESTED, CLEAN_ACTION, "003"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), - new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011"), - new HoodieInstant(REQUESTED, CLUSTERING_ACTION, "012"))), false)); + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(REQUESTED, CLEAN_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(REQUESTED, CLUSTERING_ACTION, "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR))), false)); // No inflight instants assertEquals( - Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010")), + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getEarliestInstantForMetadataArchival( prepareActiveTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), - new HoodieInstant(COMPLETED, CLEAN_ACTION, "002"), - new HoodieInstant(COMPLETED, CLEAN_ACTION, "003"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), - new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011"), - new HoodieInstant(REQUESTED, CLUSTERING_ACTION, "012"))), false)); + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, CLEAN_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, REPLACE_COMMIT_ACTION, "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(REQUESTED, CLUSTERING_ACTION, "012", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR))), false)); // Rollbacks only assertEquals( - Option.of(new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003")), + Option.of(new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getEarliestInstantForMetadataArchival( prepareActiveTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002"), - new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003"))), false)); + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(INFLIGHT, ROLLBACK_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR))), false)); assertEquals( Option.empty(), TimelineUtils.getEarliestInstantForMetadataArchival( prepareActiveTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002"), - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "003"))), false)); + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR))), false)); // With savepoints HoodieActiveTimeline timeline = prepareActiveTimeline( Arrays.asList( - new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "003"), - new HoodieInstant(COMPLETED, SAVEPOINT_ACTION, "003"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "010"), - new HoodieInstant(COMPLETED, COMMIT_ACTION, "011"))); + new HoodieInstant(COMPLETED, ROLLBACK_ACTION, "001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, SAVEPOINT_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR), + new HoodieInstant(COMPLETED, COMMIT_ACTION, "011", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR))); assertEquals( - Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "003")), + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getEarliestInstantForMetadataArchival(timeline, false)); assertEquals( - Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010")), + Option.of(new HoodieInstant(COMPLETED, COMMIT_ACTION, "010", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)), TimelineUtils.getEarliestInstantForMetadataArchival(timeline, true)); } @@ -488,7 +493,7 @@ private HoodieActiveTimeline prepareActiveTimeline( HoodieTableMetaClient mockMetaClient = mock(HoodieTableMetaClient.class); when(mockMetaClient.scanHoodieInstantsFromFileSystem(any(), eq(true))) .thenReturn(activeInstants); - return new HoodieActiveTimeline(mockMetaClient); + return new ActiveTimelineV2(mockMetaClient); } private void verifyExtraMetadataLatestValue(String extraMetadataKey, String expected, boolean includeClustering) { @@ -506,7 +511,7 @@ private byte[] getRestoreMetadata(String basePath, String partition, String comm List rollbackM = new ArrayList<>(); rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count, actionType)); List rollbackInstants = new ArrayList<>(); - rollbackInstants.add(new HoodieInstant(false, commitTs, actionType)); + rollbackInstants.add(new HoodieInstant(COMPLETED, commitTs, actionType, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)); HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(commitTs, 200, rollbackInstants, Collections.singletonMap(commitTs, rollbackM)); return TimelineMetadataUtils.serializeRestoreMetadata(metadata).get(); @@ -518,7 +523,7 @@ private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, Stri deletedFiles.add("file-" + i); } List rollbacks = new ArrayList<>(); - rollbacks.add(new HoodieInstant(false, actionType, commitTs)); + rollbacks.add(new HoodieInstant(COMPLETED, actionType, commitTs, InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR)); HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap(), Collections.emptyMap()); @@ -540,7 +545,7 @@ private byte[] getCommitMetadata(String basePath, String partition, String commi for (Map.Entry extraEntries : extraMetadata.entrySet()) { commit.addMetadata(extraEntries.getKey(), extraEntries.getValue()); } - return serializeCommitMetadata(commit).get(); + return serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commit).get(); } private byte[] getReplaceCommitMetadata(String basePath, String commitTs, String replacePartition, int replaceCount, @@ -567,7 +572,7 @@ private byte[] getReplaceCommitMetadata(String basePath, String commitTs, String for (Map.Entry extraEntries : extraMetadata.entrySet()) { commit.addMetadata(extraEntries.getKey(), extraEntries.getValue()); } - return serializeCommitMetadata(commit).get(); + return serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commit).get(); } private Option getCleanMetadata(String partition, String time, boolean isPartitionDeleted) throws IOException { @@ -638,7 +643,7 @@ public void testGetDroppedPartitions() throws Exception { String olderPartition = "p1"; // older partitions that will be deleted by clean commit // first insert to the older partition - HoodieInstant instant1 = new HoodieInstant(true, COMMIT_ACTION, "00001"); + HoodieInstant instant1 = new HoodieInstant(INFLIGHT, COMMIT_ACTION, "00001", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant1); activeTimeline.saveAsComplete(instant1, Option.of(getCommitMetadata(basePath, olderPartition, "00001", 2, Collections.emptyMap()))); @@ -648,7 +653,7 @@ public void testGetDroppedPartitions() throws Exception { assertEquals(0, droppedPartitions.size()); // another commit inserts to new partition - HoodieInstant instant2 = new HoodieInstant(true, COMMIT_ACTION, "00002"); + HoodieInstant instant2 = new HoodieInstant(INFLIGHT, COMMIT_ACTION, "00002", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(instant2); activeTimeline.saveAsComplete(instant2, Option.of(getCommitMetadata(basePath, "p2", "00002", 2, Collections.emptyMap()))); @@ -658,7 +663,7 @@ public void testGetDroppedPartitions() throws Exception { assertEquals(0, droppedPartitions.size()); // clean commit deletes older partition - HoodieInstant cleanInstant = new HoodieInstant(true, CLEAN_ACTION, "00003"); + HoodieInstant cleanInstant = new HoodieInstant(INFLIGHT, CLEAN_ACTION, "00003", InstantComparatorV2.REQUESTED_TIME_BASED_COMPARATOR); activeTimeline.createNewInstant(cleanInstant); activeTimeline.saveAsComplete(cleanInstant, getCleanMetadata(olderPartition, "00003", true)); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index f1645cd06170..c6562858b4e4 100755 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -21,7 +21,6 @@ import org.apache.hudi.common.fs.NoOpConsistencyGuard; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.MockHoodieTimeline; import org.apache.hudi.common.util.CollectionUtils; @@ -56,8 +55,14 @@ import java.util.stream.Collectors; import java.util.stream.Stream; -import static org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion.VERSION_0; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_PARSER; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -86,34 +91,36 @@ public void tearDown() throws Exception { @Test public void testLoadingInstantsFromFiles() throws IOException { - HoodieInstant instant1 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "8"); - HoodieInstant instant1Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant3Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant4Complete = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "8"); - - HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "9"); - - timeline = new HoodieActiveTimeline(metaClient); + InstantGenerator instantGenerator = INSTANT_GENERATOR; + TimelineFactory timelineFactory = TIMELINE_FACTORY; + HoodieInstant instant1 = instantGenerator.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = instantGenerator.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3 = instantGenerator.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4 = instantGenerator.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "8"); + HoodieInstant instant1Complete = instantGenerator.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2Complete = instantGenerator.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant3Complete = instantGenerator.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant4Complete = instantGenerator.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "8"); + + HoodieInstant instant5 = instantGenerator.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "9"); + + timeline = timelineFactory.createActiveTimeline(metaClient); timeline.createNewInstant(instant1); timeline.transitionRequestedToInflight(instant1, Option.empty()); // Won't lock here since InProcessLockProvider is not in hudi-common - timeline.saveAsComplete(new HoodieInstant(true, instant1.getAction(), instant1.getTimestamp()), + timeline.saveAsComplete(instantGenerator.createNewInstant(State.INFLIGHT, instant1.getAction(), instant1.requestedTime()), Option.empty()); timeline.createNewInstant(instant2); timeline.transitionRequestedToInflight(instant2, Option.empty()); - timeline.saveAsComplete(new HoodieInstant(true, instant2.getAction(), instant2.getTimestamp()), + timeline.saveAsComplete(instantGenerator.createNewInstant(State.INFLIGHT, instant2.getAction(), instant2.requestedTime()), Option.empty()); timeline.createNewInstant(instant3); timeline.transitionRequestedToInflight(instant3, Option.empty()); - timeline.saveAsComplete(new HoodieInstant(true, instant3.getAction(), instant3.getTimestamp()), + timeline.saveAsComplete(instantGenerator.createNewInstant(State.INFLIGHT, instant3.getAction(), instant3.requestedTime()), Option.empty()); timeline.createNewInstant(instant4); timeline.transitionRequestedToInflight(instant4, Option.empty()); - timeline.saveAsComplete(new HoodieInstant(true, instant4.getAction(), instant4.getTimestamp()), + timeline.saveAsComplete(instantGenerator.createNewInstant(State.INFLIGHT, instant4.getAction(), instant4.requestedTime()), Option.empty()); timeline.createNewInstant(instant5); timeline = timeline.reload(); @@ -136,46 +143,18 @@ public void testLoadingInstantsFromFiles() throws IOException { assertStreamEquals(Stream.of(instant5), timeline.getCommitAndReplaceTimeline().filterPendingExcludingCompactionAndLogCompaction().getInstantsAsStream(), "Check the instants stream"); - - // Backwards compatibility testing for reading compaction plans - metaClient = HoodieTableMetaClient.newTableBuilder() - .fromMetaClient(metaClient) - .setTimelineLayoutVersion(VERSION_0) - .initTable(metaClient.getStorageConf().newInstance(), metaClient.getBasePath()); - - HoodieInstant instant6 = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "9"); - byte[] dummy = new byte[5]; - HoodieActiveTimeline oldTimeline = new HoodieActiveTimeline( - HoodieTableMetaClient.builder().setConf(metaClient.getStorageConf().newInstance()) - .setBasePath(metaClient.getBasePath()) - .setLoadActiveTimelineOnLoad(true) - .setConsistencyGuardConfig(metaClient.getConsistencyGuardConfig()) - .setFileSystemRetryConfig(metaClient.getFileSystemRetryConfig()) - .setLayoutVersion(Option.of(new TimelineLayoutVersion(VERSION_0))).build()); - // Old Timeline writes both to aux and timeline folder - oldTimeline.saveToCompactionRequested(instant6, Option.of(dummy)); - // Now use the latest timeline version - timeline = timeline.reload(); - // Ensure aux file is present - assertTrue(metaClient.getStorage().exists(new StoragePath(metaClient.getMetaPath(), - instant6.getFileName()))); - // Read 5 bytes - assertEquals(5, timeline.readCompactionPlanAsBytes(instant6).get().length); - - // Now read compaction plan again which should not throw exception - assertEquals(5, timeline.readCompactionPlanAsBytes(instant6).get().length); } @Test public void testTimelineOperationsBasic() { - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); assertTrue(timeline.empty()); assertEquals(0, timeline.countInstants()); assertEquals(Option.empty(), timeline.firstInstant()); assertEquals(Option.empty(), timeline.nthInstant(5)); assertEquals(Option.empty(), timeline.nthInstant(-1)); assertEquals(Option.empty(), timeline.lastInstant()); - assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "01"))); + assertFalse(timeline.containsInstant(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"))); } @Test @@ -184,27 +163,27 @@ public void testTimelineOperations() { Stream.of("21", "23")); assertStreamEquals(Stream.of("05", "07", "09", "11"), timeline.getCommitAndReplaceTimeline().filterCompletedInstants().findInstantsInRange("04", "11") - .getInstantsAsStream().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::requestedTime), "findInstantsInRange should return 4 instants"); assertStreamEquals(Stream.of("03", "05", "07", "09", "11"), timeline.getCommitAndReplaceTimeline().filterCompletedInstants().findInstantsInClosedRange("03", "11") - .getInstantsAsStream().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::requestedTime), "findInstantsInClosedRange should return 5 instants"); assertStreamEquals(Stream.of("09", "11"), timeline.getCommitAndReplaceTimeline().filterCompletedInstants().findInstantsAfter("07", 2) - .getInstantsAsStream().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::requestedTime), "findInstantsAfter 07 should return 2 instants"); assertStreamEquals(Stream.of("01", "03", "05"), timeline.getCommitAndReplaceTimeline().filterCompletedInstants().findInstantsBefore("07") - .getInstantsAsStream().map(HoodieInstant::getTimestamp), + .getInstantsAsStream().map(HoodieInstant::requestedTime), "findInstantsBefore 07 should return 3 instants"); assertFalse(timeline.empty()); assertFalse(timeline.getCommitAndReplaceTimeline().filterPendingExcludingCompactionAndLogCompaction().empty()); assertEquals(12, timeline.countInstants()); assertEquals("01", timeline.firstInstant( - HoodieTimeline.COMMIT_ACTION, State.COMPLETED).get().getTimestamp()); + HoodieTimeline.COMMIT_ACTION, State.COMPLETED).get().requestedTime()); assertEquals("21", timeline.firstInstant( - HoodieTimeline.COMMIT_ACTION, State.INFLIGHT).get().getTimestamp()); + HoodieTimeline.COMMIT_ACTION, State.INFLIGHT).get().requestedTime()); assertFalse(timeline.firstInstant( HoodieTimeline.COMMIT_ACTION, State.REQUESTED).isPresent()); assertFalse(timeline.firstInstant( @@ -213,11 +192,11 @@ public void testTimelineOperations() { HoodieTimeline activeCommitTimeline = timeline.getCommitAndReplaceTimeline().filterCompletedInstants(); assertEquals(10, activeCommitTimeline.countInstants()); - assertEquals("01", activeCommitTimeline.firstInstant().get().getTimestamp()); - assertEquals("11", activeCommitTimeline.nthInstant(5).get().getTimestamp()); - assertEquals("19", activeCommitTimeline.lastInstant().get().getTimestamp()); - assertEquals("09", activeCommitTimeline.nthFromLastInstant(5).get().getTimestamp()); - assertTrue(activeCommitTimeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "09"))); + assertEquals("01", activeCommitTimeline.firstInstant().get().requestedTime()); + assertEquals("11", activeCommitTimeline.nthInstant(5).get().requestedTime()); + assertEquals("19", activeCommitTimeline.lastInstant().get().requestedTime()); + assertEquals("09", activeCommitTimeline.nthFromLastInstant(5).get().requestedTime()); + assertTrue(activeCommitTimeline.containsInstant(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "09"))); assertFalse(activeCommitTimeline.isBeforeTimelineStarts("02")); assertTrue(activeCommitTimeline.isBeforeTimelineStarts("00")); } @@ -225,19 +204,19 @@ public void testTimelineOperations() { @Test public void testAllowTempCommit() { shouldAllowTempCommit(true, hoodieMetaClient -> { - timeline = new HoodieActiveTimeline(hoodieMetaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(hoodieMetaClient); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "1"); timeline.createNewInstant(instant1); byte[] data = getUTF8Bytes("commit"); - timeline.saveAsComplete(new HoodieInstant(true, instant1.getAction(), - instant1.getTimestamp()), Option.of(data)); + timeline.saveAsComplete(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, instant1.getAction(), + instant1.requestedTime()), Option.of(data)); timeline = timeline.reload(); assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); - assertEquals(instant1.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + assertEquals(instant1.requestedTime(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().requestedTime()); }); } @@ -247,19 +226,19 @@ public void testGetContiguousCompletedWriteTimeline() { timeline = new MockHoodieTimeline(Stream.of("01", "03", "05", "07", "13", "15", "17"), Stream.of("09", "11", "19")); assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); - assertEquals("07", timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + assertEquals("07", timeline.getContiguousCompletedWriteTimeline().lastInstant().get().requestedTime()); // add some instants where two are inflight and one of them (instant8 below) is not part of write timeline - HoodieInstant instant1 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant instant2 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); - HoodieInstant instant3 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); - HoodieInstant instant4 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "4"); - HoodieInstant instant5 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, "5"); - HoodieInstant instant6 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "6"); - HoodieInstant instant7 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "7"); - HoodieInstant instant8 = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, "8"); - - timeline = new HoodieActiveTimeline(metaClient); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant instant3 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant instant4 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "4"); + HoodieInstant instant5 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "5"); + HoodieInstant instant6 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "6"); + HoodieInstant instant7 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "7"); + HoodieInstant instant8 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "8"); + + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.createCompleteInstant(instant1); timeline.createCompleteInstant(instant2); timeline.createCompleteInstant(instant3); @@ -271,24 +250,24 @@ public void testGetContiguousCompletedWriteTimeline() { timeline.setInstants(Stream.of(instant1, instant2, instant3, instant4, instant5, instant6, instant7, instant8).collect(Collectors.toList())); assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); - assertEquals(instant4.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + assertEquals(instant4.requestedTime(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().requestedTime()); // transition both inflight instants to complete - timeline.saveAsComplete(new HoodieInstant(true, instant5.getAction(), instant5.getTimestamp()), Option.empty()); - timeline.saveAsComplete(new HoodieInstant(true, instant8.getAction(), instant8.getTimestamp()), Option.empty()); + timeline.saveAsComplete(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, instant5.getAction(), instant5.requestedTime()), Option.empty()); + timeline.saveAsComplete(INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, instant8.getAction(), instant8.requestedTime()), Option.empty()); timeline = timeline.reload(); // instant8 in not considered in write timeline, so last completed instant in timeline should be instant7 assertTrue(timeline.getContiguousCompletedWriteTimeline().lastInstant().isPresent()); - assertEquals(instant7.getTimestamp(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().getTimestamp()); + assertEquals(instant7.requestedTime(), timeline.getContiguousCompletedWriteTimeline().lastInstant().get().requestedTime()); } @Test public void testTimelineWithSavepointAndHoles() { timeline = new MockHoodieTimeline(Stream.of( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") // this can be DELTA_COMMIT/REPLACE_COMMIT as well ).collect(Collectors.toList())); assertTrue(timeline.isBeforeTimelineStarts("00")); assertTrue(timeline.isBeforeTimelineStarts("01")); @@ -300,11 +279,11 @@ public void testTimelineWithSavepointAndHoles() { // with an inflight savepoint in between timeline = new MockHoodieTimeline(Stream.of( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, "01"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.SAVEPOINT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05") ).collect(Collectors.toList())); assertTrue(timeline.isBeforeTimelineStarts("00")); assertTrue(timeline.isBeforeTimelineStarts("01")); @@ -316,13 +295,13 @@ public void testTimelineWithSavepointAndHoles() { // with a pending replacecommit after savepoints timeline = new MockHoodieTimeline(Stream.of( - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), - new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, "06"), - new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, "07") + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, "06"), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, "07") ).collect(Collectors.toList())); assertTrue(timeline.isBeforeTimelineStarts("00")); assertTrue(timeline.isBeforeTimelineStarts("01")); @@ -337,7 +316,7 @@ public void testTimelineWithSavepointAndHoles() { public void testTimelineGetOperations() { List allInstants = getAllInstants(); Supplier> allInstantsSup = allInstants::stream; - timeline = new HoodieActiveTimeline(metaClient, true); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, true); timeline.setInstants(allInstants); /* @@ -382,10 +361,10 @@ public void testTimelineGetOperations() { @Test public void testTimelineInstantOperations() { - timeline = new HoodieActiveTimeline(metaClient, true); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, true); assertEquals(0, timeline.countInstants(), "No instant present"); // revertToInflight - HoodieInstant commit = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant commit = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); timeline.createCompleteInstant(commit); timeline = timeline.reload(); assertEquals(1, timeline.countInstants()); @@ -417,7 +396,7 @@ public void testTimelineInstantOperations() { assertTrue(timeline.containsInstant(commit)); // deleteCompactionRequested - HoodieInstant compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2"); + HoodieInstant compaction = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "2"); timeline.createNewInstant(compaction); timeline = timeline.reload(); assertEquals(2, timeline.countInstants()); @@ -429,7 +408,7 @@ public void testTimelineInstantOperations() { assertTrue(timeline.containsInstant(commit)); // transitionCompactionXXXtoYYY and revertCompactionXXXtoYYY - compaction = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); + compaction = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); timeline.createNewInstant(compaction); timeline = timeline.reload(); assertTrue(timeline.containsInstant(compaction)); @@ -448,7 +427,7 @@ public void testTimelineInstantOperations() { assertFalse(timeline.containsInstant(inflight)); // transitionCleanXXXtoYYY - HoodieInstant clean = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4"); + HoodieInstant clean = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "4"); timeline.saveToCleanRequested(clean, Option.empty()); timeline = timeline.reload(); assertTrue(timeline.containsInstant(clean)); @@ -462,26 +441,26 @@ public void testTimelineInstantOperations() { assertFalse(timeline.containsInstant(inflight)); // Various states of Instants - HoodieInstant srcInstant = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5"); - HoodieInstant otherInstant = HoodieTimeline.getRequestedInstant(srcInstant); - assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5")); - otherInstant = HoodieTimeline.getCleanRequestedInstant("5"); - assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5")); - otherInstant = HoodieTimeline.getCleanInflightInstant("5"); - assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5")); - otherInstant = HoodieTimeline.getCompactionRequestedInstant("5"); - assertEquals(otherInstant, new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5")); - otherInstant = HoodieTimeline.getCompactionInflightInstant("5"); - assertEquals(otherInstant, new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5")); + HoodieInstant srcInstant = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5"); + HoodieInstant otherInstant = INSTANT_GENERATOR.getRequestedInstant(srcInstant); + assertEquals(otherInstant, INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5")); + otherInstant = INSTANT_GENERATOR.getCleanRequestedInstant("5"); + assertEquals(otherInstant, INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "5")); + otherInstant = INSTANT_GENERATOR.getCleanInflightInstant("5"); + assertEquals(otherInstant, INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, "5")); + otherInstant = INSTANT_GENERATOR.getCompactionRequestedInstant("5"); + assertEquals(otherInstant, INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5")); + otherInstant = INSTANT_GENERATOR.getCompactionInflightInstant("5"); + assertEquals(otherInstant, INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "5")); // containsOrBeforeTimelineStarts List allInstants = getAllInstants(); - timeline = new HoodieActiveTimeline(metaClient, true); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient, true); timeline.setInstants(allInstants); timeline.setInstants(allInstants); - timeline.createNewInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2")); - allInstants.stream().map(HoodieInstant::getTimestamp).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s))); + timeline.createNewInstant(INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "2")); + allInstants.stream().map(HoodieInstant::requestedTime).forEach(s -> assertTrue(timeline.containsOrBeforeTimelineStarts(s))); assertTrue(timeline.containsOrBeforeTimelineStarts("0")); assertFalse(timeline.containsOrBeforeTimelineStarts(String.valueOf(System.currentTimeMillis() + 1000))); assertFalse(timeline.getTimelineHash().isEmpty()); @@ -506,16 +485,16 @@ public void testCreateInstants() { @Test public void testInstantFilenameOperations() { - HoodieInstant instantRequested = new HoodieInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"); - HoodieInstant instantInflight = new HoodieInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5"); - HoodieInstant instantComplete = new HoodieInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5", "6"); - assertEquals(HoodieTimeline.getCommitFromCommitFile(instantRequested.getFileName()), "5"); - assertEquals(HoodieTimeline.getCommitFromCommitFile(instantInflight.getFileName()), "5"); - assertEquals(HoodieTimeline.getCommitFromCommitFile(instantComplete.getFileName()), "5_6"); - - assertEquals(HoodieTimeline.makeInflightRestoreFileName( - HoodieInstant.extractTimestamp(instantComplete.getFileName())), - instantInflight.getFileName()); + HoodieInstant instantRequested = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.RESTORE_ACTION, "5"); + HoodieInstant instantInflight = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, "5"); + HoodieInstant instantComplete = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.RESTORE_ACTION, "5", "6"); + assertEquals(INSTANT_FILE_NAME_GENERATOR.getCommitFromCommitFile(INSTANT_FILE_NAME_GENERATOR.getFileName(instantRequested)), "5"); + assertEquals(INSTANT_FILE_NAME_GENERATOR.getCommitFromCommitFile(INSTANT_FILE_NAME_GENERATOR.getFileName(instantInflight)), "5"); + assertEquals(INSTANT_FILE_NAME_GENERATOR.getCommitFromCommitFile(INSTANT_FILE_NAME_GENERATOR.getFileName(instantComplete)), "5_6"); + + assertEquals(INSTANT_FILE_NAME_GENERATOR.makeInflightRestoreFileName( + INSTANT_FILE_NAME_PARSER.extractTimestamp(INSTANT_FILE_NAME_GENERATOR.getFileName(instantComplete))), + INSTANT_FILE_NAME_GENERATOR.getFileName(instantInflight)); } @Test @@ -523,7 +502,7 @@ public void testFiltering() { List allInstants = getAllInstants(); Supplier> sup = allInstants::stream; - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(allInstants); // getReverseOrderedInstants @@ -570,20 +549,20 @@ public void testFiltering() { public void testReplaceActionsTimeline() { int instantTime = 1; List allInstants = new ArrayList<>(); - HoodieInstant instant1 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); allInstants.add(instant1); - HoodieInstant instant2 = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); allInstants.add(instant2); - HoodieInstant instant3 = new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, String.format("%03d", instantTime++)); + HoodieInstant instant3 = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, String.format("%03d", instantTime++)); allInstants.add(instant3); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(allInstants); List validReplaceInstants = timeline.getCompletedReplaceTimeline().getInstants(); assertEquals(1, validReplaceInstants.size()); - assertEquals(instant3.getTimestamp(), validReplaceInstants.get(0).getTimestamp()); + assertEquals(instant3.requestedTime(), validReplaceInstants.get(0).requestedTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, validReplaceInstants.get(0).getAction()); assertStreamEquals( @@ -600,7 +579,7 @@ public void testCreateNewInstantTime() throws Exception { String lastInstantTime = metaClient.createNewInstantTime(false); for (int i = 0; i < 3; ++i) { String newInstantTime = metaClient.createNewInstantTime(false); - assertTrue(HoodieTimeline.compareTimestamps(lastInstantTime, HoodieTimeline.LESSER_THAN, newInstantTime)); + assertTrue(compareTimestamps(lastInstantTime, LESSER_THAN, newInstantTime)); lastInstantTime = newInstantTime; } @@ -613,9 +592,9 @@ public void testCreateNewInstantTime() throws Exception { for (int idx = 0; idx < numThreads; ++idx) { futures.add(executorService.submit(() -> { Date date = new Date(System.currentTimeMillis() + (int)(Math.random() * numThreads) * milliSecondsInYear); - final String expectedFormat = HoodieActiveTimeline.formatDate(date); + final String expectedFormat = TimelineUtils.formatDate(date); for (int tidx = 0; tidx < numChecks; ++tidx) { - final String curFormat = HoodieActiveTimeline.formatDate(date); + final String curFormat = TimelineUtils.formatDate(date); if (!curFormat.equals(expectedFormat)) { throw new HoodieException("Format error: expected=" + expectedFormat + ", curFormat=" + curFormat); } @@ -635,18 +614,18 @@ public void testCreateNewInstantTime() throws Exception { public void testMinTimestamp() { String timestamp1 = "20240601040632402"; String timestamp2 = "20250601040632402"; - assertEquals(timestamp1, HoodieTimeline.minTimestamp(null, timestamp1)); - assertEquals(timestamp1, HoodieTimeline.minTimestamp("", timestamp1)); - assertEquals(timestamp1, HoodieTimeline.minTimestamp(timestamp1, null)); - assertEquals(timestamp1, HoodieTimeline.minTimestamp(timestamp1, "")); - assertEquals(timestamp1, HoodieTimeline.minTimestamp(timestamp1, timestamp2)); + assertEquals(timestamp1, InstantComparison.minTimestamp(null, timestamp1)); + assertEquals(timestamp1, InstantComparison.minTimestamp("", timestamp1)); + assertEquals(timestamp1, InstantComparison.minTimestamp(timestamp1, null)); + assertEquals(timestamp1, InstantComparison.minTimestamp(timestamp1, "")); + assertEquals(timestamp1, InstantComparison.minTimestamp(timestamp1, timestamp2)); } @Test public void testParseDateFromInstantTime() throws ParseException { // default second granularity instant ID String secondGranularityInstant = "20210101120101123"; - Date defaultSecsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant); + Date defaultSecsGranularityDate = TimelineUtils.parseDateFromInstantTime(secondGranularityInstant); System.out.println(defaultSecsGranularityDate.getTime()); } @@ -654,23 +633,23 @@ public void testParseDateFromInstantTime() throws ParseException { public void testMetadataCompactionInstantDateParsing() throws ParseException { // default second granularity instant ID String secondGranularityInstant = "20210101120101123"; - Date defaultSecsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant); + Date defaultSecsGranularityDate = TimelineUtils.parseDateFromInstantTime(secondGranularityInstant); // metadata table compaction/cleaning : ms granularity instant ID String compactionInstant = secondGranularityInstant + "001"; - Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(compactionInstant); + Date defaultMsGranularityDate = TimelineUtils.parseDateFromInstantTime(compactionInstant); assertEquals(0, defaultMsGranularityDate.getTime() - defaultSecsGranularityDate.getTime(), "Expected the ms part to be 0"); - assertTrue(HoodieTimeline.compareTimestamps(secondGranularityInstant, HoodieTimeline.LESSER_THAN, compactionInstant)); - assertTrue(HoodieTimeline.compareTimestamps(compactionInstant, HoodieTimeline.GREATER_THAN, secondGranularityInstant)); + assertTrue(InstantComparison.compareTimestamps(secondGranularityInstant, LESSER_THAN, compactionInstant)); + assertTrue(InstantComparison.compareTimestamps(compactionInstant, GREATER_THAN, secondGranularityInstant)); } @Test public void testMillisGranularityInstantDateParsing() throws ParseException { // Old second granularity instant ID String secondGranularityInstant = "20210101120101"; - Date defaultMsGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant); + Date defaultMsGranularityDate = TimelineUtils.parseDateFromInstantTime(secondGranularityInstant); // New ms granularity instant ID String specificMsGranularityInstant = secondGranularityInstant + "009"; - Date msGranularityDate = HoodieActiveTimeline.parseDateFromInstantTime(specificMsGranularityInstant); + Date msGranularityDate = TimelineUtils.parseDateFromInstantTime(specificMsGranularityInstant); assertEquals(999, defaultMsGranularityDate.getTime() % 1000, "Expected the ms part to be 999"); assertEquals(9, msGranularityDate.getTime() % 1000, "Expected the ms part to be 9"); @@ -678,8 +657,8 @@ public void testMillisGranularityInstantDateParsing() throws ParseException { String laterDateInstant = "20210101120111"; // + 10 seconds from original instant assertEquals( 10, - HoodieActiveTimeline.parseDateFromInstantTime(laterDateInstant).getTime() / 1000 - - HoodieActiveTimeline.parseDateFromInstantTime(secondGranularityInstant).getTime() / 1000, + TimelineUtils.parseDateFromInstantTime(laterDateInstant).getTime() / 1000 + - TimelineUtils.parseDateFromInstantTime(secondGranularityInstant).getTime() / 1000, "Expected the difference between later instant and previous instant to be 10 seconds" ); } @@ -688,36 +667,36 @@ public void testMillisGranularityInstantDateParsing() throws ParseException { public void testInvalidInstantDateParsing() throws ParseException { // Test all invalid timestamp in HoodieTimeline, shouldn't throw any error and should return a correct value assertEquals(Long.parseLong(HoodieTimeline.INIT_INSTANT_TS), - HoodieActiveTimeline.parseDateFromInstantTimeSafely(HoodieTimeline.INIT_INSTANT_TS).get().getTime()); + TimelineUtils.parseDateFromInstantTimeSafely(HoodieTimeline.INIT_INSTANT_TS).get().getTime()); assertEquals(Long.parseLong(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS), - HoodieActiveTimeline.parseDateFromInstantTimeSafely(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).get().getTime()); + TimelineUtils.parseDateFromInstantTimeSafely(HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS).get().getTime()); assertEquals(Long.parseLong(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS), - HoodieActiveTimeline.parseDateFromInstantTimeSafely(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS).get().getTime()); + TimelineUtils.parseDateFromInstantTimeSafely(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS).get().getTime()); // Test metadata table compaction instant date parsing with INIT_INSTANT_TS, should return Option.empty assertEquals(Option.empty(), - HoodieActiveTimeline.parseDateFromInstantTimeSafely(HoodieTimeline.INIT_INSTANT_TS + "001")); + TimelineUtils.parseDateFromInstantTimeSafely(HoodieTimeline.INIT_INSTANT_TS + "001")); - // Test a valid instant timestamp, should equal the same result as HoodieActiveTimeline.parseDateFromInstantTime + // Test a valid instant timestamp, should equal the same result as ActiveTimelineUtils.parseDateFromInstantTime String testInstant = "20210101120101"; - assertEquals(HoodieActiveTimeline.parseDateFromInstantTime(testInstant).getTime(), - HoodieActiveTimeline.parseDateFromInstantTimeSafely(testInstant).get().getTime()); + assertEquals(TimelineUtils.parseDateFromInstantTime(testInstant).getTime(), + TimelineUtils.parseDateFromInstantTimeSafely(testInstant).get().getTime()); } @Test public void testInstantCompletionTimeBackwardCompatibility() { - HoodieInstant requestedInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "2"); - HoodieInstant completeInstant = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); + HoodieInstant requestedInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant inflightInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, "2"); + HoodieInstant completeInstant = INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "3"); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.createNewInstant(requestedInstant); timeline.createNewInstant(inflightInstant); // Note: // 0.x meta file name pattern: ${instant_time}.action[.state] // 1.x meta file name pattern: ${instant_time}_${completion_time}.action[.state]. - String legacyCompletedFileName = HoodieTimeline.makeCommitFileName(completeInstant.getTimestamp()); + String legacyCompletedFileName = INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(completeInstant.requestedTime()); metaClient.getStorage().createImmutableFileInPath(new StoragePath(metaClient.getMetaPath().toString(), legacyCompletedFileName), Option.empty()); timeline = timeline.reload(); @@ -726,7 +705,7 @@ public void testInstantCompletionTimeBackwardCompatibility() { assertNull(instants.get(0).getCompletionTime(), "Requested instant does not have completion time"); assertNull(instants.get(1).getCompletionTime(), "Inflight instant does not have completion time"); assertNotNull(instants.get(2).getCompletionTime(), "Completed instant has modification time as completion time for 0.x release"); - assertEquals(instants.get(2).getTimestamp() + HoodieTimeline.COMMIT_EXTENSION, instants.get(2).getFileName(), "Instant file name should not have completion time"); + assertEquals(instants.get(2).requestedTime() + HoodieTimeline.COMMIT_EXTENSION, INSTANT_FILE_NAME_GENERATOR.getFileName(instants.get(2)), "Instant file name should not have completion time"); } /** @@ -734,7 +713,7 @@ public void testInstantCompletionTimeBackwardCompatibility() { * @return list of HoodieInstant */ private List getAllInstants() { - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); List allInstants = new ArrayList<>(); long instantTime = 1; for (State state : State.values()) { @@ -762,7 +741,7 @@ private List getAllInstants() { action = HoodieTimeline.REPLACE_COMMIT_ACTION; } - allInstants.add(new HoodieInstant(state, action, String.format("%03d", instantTime++))); + allInstants.add(INSTANT_GENERATOR.createNewInstant(state, action, String.format("%03d", instantTime++))); } } return allInstants; diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java index a3f2fc8de956..bb6510df6bb4 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieInstant.java @@ -28,6 +28,9 @@ import java.util.stream.Stream; import static org.apache.hudi.common.testutils.Assertions.assertStreamEquals; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_PARSER; + import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -37,31 +40,31 @@ public class TestHoodieInstant extends HoodieCommonTestHarness { @Test public void testExtractTimestamp() { String fileName = "20230104152218702.inflight"; - assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName)); + assertEquals("20230104152218702", INSTANT_FILE_NAME_PARSER.extractTimestamp(fileName)); fileName = "20230104152218702.commit.request"; - assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName)); + assertEquals("20230104152218702", INSTANT_FILE_NAME_PARSER.extractTimestamp(fileName)); fileName = "20230104152218702_20230104152219346.commit"; - assertEquals("20230104152218702", HoodieInstant.extractTimestamp(fileName)); + assertEquals("20230104152218702", INSTANT_FILE_NAME_PARSER.extractTimestamp(fileName)); String illegalFileName = "hoodie.properties"; - assertThrows(IllegalArgumentException.class, () -> HoodieInstant.extractTimestamp(illegalFileName)); + assertThrows(IllegalArgumentException.class, () -> INSTANT_FILE_NAME_PARSER.extractTimestamp(illegalFileName)); } @Test public void testGetTimelineFileExtension() { String fileName = "20230104152218702.inflight"; - assertEquals(".inflight", HoodieInstant.getTimelineFileExtension(fileName)); + assertEquals(".inflight", INSTANT_FILE_NAME_PARSER.getTimelineFileExtension(fileName)); fileName = "20230104152218702.commit.request"; - assertEquals(".commit.request", HoodieInstant.getTimelineFileExtension(fileName)); + assertEquals(".commit.request", INSTANT_FILE_NAME_PARSER.getTimelineFileExtension(fileName)); fileName = "20230104152218702_20230104152219346.commit"; - assertEquals(".commit", HoodieInstant.getTimelineFileExtension(fileName)); + assertEquals(".commit", INSTANT_FILE_NAME_PARSER.getTimelineFileExtension(fileName)); fileName = "hoodie.properties"; - assertEquals("", HoodieInstant.getTimelineFileExtension(fileName)); + assertEquals("", INSTANT_FILE_NAME_PARSER.getTimelineFileExtension(fileName)); } @Test @@ -69,14 +72,14 @@ public void testCreateHoodieInstantByFileStatus() throws IOException { try { initMetaClient(); HoodieInstant instantRequested = - new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "001"); HoodieInstant instantCommitted = - new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "001"); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); timeline.createNewInstant(instantRequested); timeline.transitionRequestedToInflight(instantRequested, Option.empty()); timeline.saveAsComplete( - new HoodieInstant(true, instantRequested.getAction(), instantRequested.getTimestamp()), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instantRequested.getAction(), instantRequested.requestedTime()), Option.empty()); metaClient.reloadActiveTimeline(); timeline = metaClient.getActiveTimeline(); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFSViewWithClustering.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFSViewWithClustering.java index 4ff5ce2a888f..1ceea1c6b7f2 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFSViewWithClustering.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFSViewWithClustering.java @@ -48,6 +48,7 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -145,11 +146,11 @@ public void testReplaceFileIdIsExcludedInView() throws IOException { CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, commitTime1); saveAsComplete( commitTimeline, instant1, - serializeCommitMetadata(commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); refreshFsView(); assertEquals(0, roView.getLatestBaseFiles(partitionPath1) .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); @@ -184,7 +185,7 @@ private void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant infligh if (inflight.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { timeline.transitionCompactionInflightToComplete(true, inflight, data); } else { - HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, inflight.getAction(), inflight.getTimestamp()); + HoodieInstant requested = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, inflight.getAction(), inflight.requestedTime()); timeline.createNewInstant(requested); timeline.transitionRequestedToInflight(requested, Option.empty()); if (inflight.getAction().equals(HoodieTimeline.CLUSTERING_ACTION)) { diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index a9b746169a6c..d3c3fa0e2300 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -94,6 +94,7 @@ import static org.apache.hudi.common.model.HoodiePartitionMetadata.HOODIE_PARTITION_METAFILE_PREFIX; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; @@ -179,12 +180,12 @@ public void testCloseHoodieTableFileSystemView() throws Exception { partitionToReplaceFileIds.put("fake_partition_path", replacedFileIds); // prepare Instants - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime2); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime2); HoodieInstant clusteringInstant3 = - new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime3); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime3); HoodieInstant clusteringInstant4 = - new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime4); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime4); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.CLUSTER, "", HoodieTimeline.REPLACE_COMMIT_ACTION); @@ -192,11 +193,11 @@ public void testCloseHoodieTableFileSystemView() throws Exception { saveAsComplete(commitTimeline, instant1, Option.empty()); saveAsComplete(commitTimeline, instant2, Option.empty()); saveAsCompleteCluster(commitTimeline, clusteringInstant3, - serializeCommitMetadata((HoodieReplaceCommitMetadata) commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), (HoodieReplaceCommitMetadata) commitMetadata)); saveAsCompleteCluster( commitTimeline, clusteringInstant4, - serializeCommitMetadata((HoodieReplaceCommitMetadata) commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), (HoodieReplaceCommitMetadata) commitMetadata)); refreshFsView(); @@ -235,9 +236,9 @@ protected void testViewForFileSlicesWithNoBaseFile(int expNumTotalFileSlices, in Paths.get(basePath, partitionPath, fileName2).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); - HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); saveAsComplete(commitTimeline, instant1, Option.empty()); saveAsComplete(commitTimeline, deltaInstant2, Option.empty()); @@ -332,7 +333,7 @@ public void testViewForFileSlicesWithPartitionMetadataFile() throws Exception { new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); @@ -365,7 +366,7 @@ public void testViewForGetAllFileGroupsStateless() throws Exception { new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); // create 2 fileId in partition2 @@ -377,7 +378,7 @@ public void testViewForGetAllFileGroupsStateless() throws Exception { new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime2); saveAsComplete(commitTimeline, instant2, Option.empty()); fsView.sync(); @@ -424,7 +425,7 @@ public void testViewForGetLatestFileSlicesStateless() throws Exception { new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); // create 2 fileId in partition2 @@ -436,7 +437,7 @@ public void testViewForGetLatestFileSlicesStateless() throws Exception { new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime2); saveAsComplete(commitTimeline, instant2, Option.empty()); fsView.sync(); @@ -487,9 +488,9 @@ protected void testInvalidLogFiles() throws Exception { Paths.get(basePath, partitionPath, fileName4).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); - HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); saveAsComplete(commitTimeline, instant1, Option.empty()); saveAsComplete(commitTimeline, deltaInstant2, Option.empty()); @@ -563,10 +564,10 @@ void testFileSlicingWithMultipleDeltaWriters() throws Exception { HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant deltaInstant1 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); - HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); - HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime3); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + HoodieInstant deltaInstant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime3); // delta instant 3 starts but finishes in the last metaClient.getActiveTimeline().createNewInstant(deltaInstant3); @@ -596,8 +597,8 @@ void testFileSlicingWithMultipleDeltaWriters() throws Exception { CompactionUtils.buildFromFileSlices(partitionFileSlicesPairs, Option.empty(), Option.empty()); // Create a Data-file but this should be skipped by view Paths.get(basePath, partitionPath, compactionFile1).toFile().createNewFile(); - HoodieInstant compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime1); - HoodieInstant requested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + HoodieInstant compactionInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime1); + HoodieInstant requested = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstant.requestedTime()); commitTimeline.saveToCompactionRequested(requested, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); commitTimeline.transitionCompactionRequestedToInflight(requested); @@ -678,7 +679,7 @@ void testLoadPartitions_unPartitioned() throws Exception { Paths.get(basePath, partitionPath, fileName1).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); @@ -708,7 +709,7 @@ void testLoadPartitions_partitioned() throws Exception { Paths.get(basePath, partitionPath1, fileName1).toFile().createNewFile(); Paths.get(basePath, partitionPath2, fileName2).toFile().createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); @@ -781,9 +782,9 @@ public void testGetLatestFileSlicesIncludingInflight() throws Exception { new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); - HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); // just commit instant1 and deltaInstant2, keep deltaInstant3 inflight saveAsComplete(commitTimeline, instant1, Option.empty()); @@ -868,9 +869,9 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); - HoodieInstant deltaInstant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); - HoodieInstant deltaInstant3 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant deltaInstant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + HoodieInstant deltaInstant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); if (testBootstrap && !skipCreatingDataFile) { try (IndexWriter writer = new HFileBootstrapIndex(metaClient).createWriter(BOOTSTRAP_SOURCE_PATH)) { @@ -910,15 +911,15 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData if (isCompactionInFlight) { // Create a Data-file but this should be skipped by view new File(basePath + "/" + partitionPath + "/" + compactDataFileName).createNewFile(); - compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, + compactionInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); HoodieInstant requested = - HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstant.requestedTime()); commitTimeline.saveToCompactionRequested(requested, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); commitTimeline.transitionCompactionRequestedToInflight(requested); } else { - compactionInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + compactionInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); commitTimeline.saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); } @@ -942,8 +943,8 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData FSUtils.makeLogFileName(fileId, HoodieLogFile.DELTA_EXTENSION, deltaInstantTime5, 1, TEST_WRITE_TOKEN); new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile(); - HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); - HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); + HoodieInstant deltaInstant4 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); + HoodieInstant deltaInstant5 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); saveAsComplete(commitTimeline, deltaInstant4, Option.empty()); saveAsComplete(commitTimeline, deltaInstant5, Option.empty()); refreshFsView(); @@ -1053,10 +1054,10 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData new File(basePath + "/" + partitionPath + "/" + inflightLogFileName).createNewFile(); // Mark instant as inflight commitTimeline.createNewInstant( - new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, inflightDeltaInstantTime)); commitTimeline.transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, inflightDeltaInstantTime), Option.empty()); refreshFsView(); @@ -1127,7 +1128,7 @@ protected void testViewForFileSlicesWithAsyncCompaction(boolean skipCreatingData "Inflight File Slice with log-file check data-file"); } - compactionInstant = new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); + compactionInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); // Now simulate Compaction completing - Check the view if (!isCompactionInFlight) { // For inflight compaction, we already create a data-file to test concurrent inflight case. @@ -1216,7 +1217,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { // Make this commit safe HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); assertEquals(fileName1, roView.getLatestBaseFiles(partitionPath) @@ -1231,7 +1232,7 @@ public void testGetLatestDataFilesForFileId() throws IOException { .filter(dfile -> dfile.getFileId().equals(fileId)).findFirst().get().getFileName()); // Make it safe - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime2); saveAsComplete(commitTimeline, instant2, Option.empty()); refreshFsView(); assertEquals(fileName2, roView.getLatestBaseFiles(partitionPath) @@ -1711,11 +1712,11 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E new File(fullPartitionPath3 + fileName1).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, instantTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, instantTime1); HoodieInstant deltaInstant2 = - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime1); HoodieInstant deltaInstant3 = - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime2); saveAsComplete(commitTimeline, instant1, Option.empty()); saveAsComplete(commitTimeline, deltaInstant2, Option.empty()); @@ -1765,9 +1766,9 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E new File(basePath + "/" + partitionPath2 + "/" + compactDataFileName).createNewFile(); HoodieInstant compactionInstant = - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionRequestedTime); - HoodieInstant requested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + HoodieInstant requested = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstant.requestedTime()); metaClient.getActiveTimeline().saveToCompactionRequested(requested, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(requested); @@ -1786,8 +1787,8 @@ public void testPendingCompactionWithDuplicateFileIdsAcrossPartitions() throws E new File(basePath + "/" + partitionPath3 + "/" + fileName3).createNewFile(); new File(basePath + "/" + partitionPath3 + "/" + fileName4).createNewFile(); - HoodieInstant deltaInstant4 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); - HoodieInstant deltaInstant5 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); + HoodieInstant deltaInstant4 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime4); + HoodieInstant deltaInstant5 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, deltaInstantTime5); saveAsComplete(commitTimeline, deltaInstant4, Option.empty()); saveAsComplete(commitTimeline, deltaInstant5, Option.empty()); refreshFsView(); @@ -1853,7 +1854,7 @@ private void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant infligh if (inflight.getAction().equals(HoodieTimeline.COMPACTION_ACTION)) { timeline.transitionCompactionInflightToComplete(true, inflight, data); } else { - HoodieInstant requested = new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()); + HoodieInstant requested = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, inflight.getAction(), inflight.requestedTime()); timeline.createNewInstant(requested); timeline.transitionRequestedToInflight(requested, Option.empty()); timeline.saveAsComplete(inflight, data); @@ -1862,7 +1863,7 @@ private void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant infligh private void saveAsCompleteCluster(HoodieActiveTimeline timeline, HoodieInstant inflight, Option data) { assertEquals(HoodieTimeline.CLUSTERING_ACTION, inflight.getAction()); - HoodieInstant clusteringInstant = new HoodieInstant(State.REQUESTED, inflight.getAction(), inflight.getTimestamp()); + HoodieInstant clusteringInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, inflight.getAction(), inflight.requestedTime()); HoodieClusteringPlan plan = new HoodieClusteringPlan(); plan.setExtraMetadata(new HashMap<>()); plan.setInputGroups(Collections.emptyList()); @@ -1904,7 +1905,7 @@ public void testReplaceWithTimeTravel() throws IOException { new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); assertEquals(1, roView.getLatestBaseFiles(partitionPath1) @@ -1930,11 +1931,11 @@ public void testReplaceWithTimeTravel() throws IOException { Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2); saveAsComplete( commitTimeline, instant2, - serializeCommitMetadata(commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); //make sure view doesn't include fileId1 refreshFsView(); @@ -2020,11 +2021,11 @@ public void testReplaceFileIdIsExcludedInView() throws IOException { CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime1); saveAsComplete( commitTimeline, instant1, - serializeCommitMetadata(commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); refreshFsView(); assertEquals(0, roView.getLatestBaseFiles(partitionPath1) .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); @@ -2071,7 +2072,7 @@ public void testPendingClusteringOperations() throws IOException { new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); saveAsComplete(commitTimeline, instant1, Option.empty()); refreshFsView(); assertEquals(1, roView.getLatestBaseFiles(partitionPath1) @@ -2091,7 +2092,7 @@ public void testPendingClusteringOperations() throws IOException { fileSliceGroups, Collections.emptyMap()); String clusterTime = "2"; - HoodieInstant instant2 = new HoodieInstant(State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusterTime); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusterTime); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setClusteringPlan(plan).setOperationType(WriteOperationType.CLUSTER.name()).build(); metaClient.getActiveTimeline().saveToPendingClusterCommit(instant2, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); @@ -2182,7 +2183,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept new File(basePath + "/" + partitionPath + "/" + fileName1).createNewFile(); new File(basePath + "/" + partitionPath + "/" + fileName2).createNewFile(); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime1); // build writeStats HashMap> partitionToFile1 = new HashMap<>(); @@ -2197,7 +2198,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept saveAsComplete( commitTimeline, instant1, - serializeCommitMetadata(commitMetadata1)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata1)); commitTimeline.reload(); // replace commit @@ -2205,7 +2206,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept String fileName3 = FSUtils.makeBaseFileName(commitTime2, TEST_WRITE_TOKEN, fileId3, BASE_FILE_EXTENSION); new File(basePath + "/" + partitionPath + "/" + fileName3).createNewFile(); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, commitTime2); Map> partitionToReplaceFileIds = new HashMap<>(); List replacedFileIds = new ArrayList<>(); replacedFileIds.add(fileId1); @@ -2223,13 +2224,13 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept saveAsCompleteCluster( commitTimeline, instant2, - serializeCommitMetadata(commitMetadata2)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata2)); // another insert commit String commitTime3 = "3"; String fileName4 = FSUtils.makeBaseFileName(commitTime3, TEST_WRITE_TOKEN, fileId4, BASE_FILE_EXTENSION); new File(basePath + "/" + partitionPath + "/" + fileName4).createNewFile(); - HoodieInstant instant3 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime3); + HoodieInstant instant3 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, commitTime3); // build writeStats HashMap> partitionToFile3 = new HashMap<>(); @@ -2242,7 +2243,7 @@ public void testHoodieTableFileSystemViewWithPendingClustering() throws IOExcept saveAsComplete( commitTimeline, instant3, - serializeCommitMetadata(commitMetadata3)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata3)); metaClient.reloadActiveTimeline(); refreshFsView(); @@ -2380,11 +2381,11 @@ public void testPendingMajorAndMinorCompactionOperations() throws Exception { HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId1, logFileName1)); commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId2, logFileName2)); - HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime1); + HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime1); saveAsComplete( commitTimeline, instant1, - serializeCommitMetadata(commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); SyncableFileSystemView fileSystemView = getFileSystemView(metaClient.reloadActiveTimeline(), true); @@ -2401,12 +2402,12 @@ public void testPendingMajorAndMinorCompactionOperations() throws Exception { commitTimeline = metaClient.getActiveTimeline(); commitMetadata = new HoodieCommitMetadata(); commitMetadata.addWriteStat(partitionPath, getHoodieWriteStat(partitionPath, fileId1, logFileName3)); - HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime2); + HoodieInstant instant2 = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, commitTime2); saveAsComplete( commitTimeline, instant2, - serializeCommitMetadata(commitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); // Verify file system view after 2nd commit. verifyFileSystemView(partitionPath, expectedState, fileSystemView); @@ -2422,8 +2423,8 @@ public void testPendingMajorAndMinorCompactionOperations() throws Exception { String commitTime3 = "003"; HoodieInstant compactionInstant = - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, commitTime3); - HoodieInstant compactionRequested = HoodieTimeline.getCompactionRequestedInstant(compactionInstant.getTimestamp()); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, commitTime3); + HoodieInstant compactionRequested = INSTANT_GENERATOR.getCompactionRequestedInstant(compactionInstant.requestedTime()); metaClient.getActiveTimeline().saveToCompactionRequested(compactionRequested, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); metaClient.getActiveTimeline().transitionCompactionRequestedToInflight(compactionRequested); @@ -2440,8 +2441,8 @@ public void testPendingMajorAndMinorCompactionOperations() throws Exception { String commitTime4 = "004"; HoodieInstant logCompactionInstant = - new HoodieInstant(State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, commitTime4); - HoodieInstant logCompactionRequested = HoodieTimeline.getLogCompactionRequestedInstant(logCompactionInstant.getTimestamp()); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.LOG_COMPACTION_ACTION, commitTime4); + HoodieInstant logCompactionRequested = INSTANT_GENERATOR.getLogCompactionRequestedInstant(logCompactionInstant.requestedTime()); metaClient.getActiveTimeline().saveToLogCompactionRequested(logCompactionRequested, TimelineMetadataUtils.serializeCompactionPlan(logCompactionPlan)); metaClient.getActiveTimeline().transitionLogCompactionRequestedToInflight(logCompactionRequested); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 2838b10e3190..9d79bdb0cd30 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -42,6 +42,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; @@ -79,7 +80,11 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.LOG_COMPACTION_ACTION; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createMetaClient; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -171,7 +176,7 @@ public void testAsyncCompaction() throws IOException { // Finish Compaction instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("19"), false, "19", 2, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); view.close(); } @@ -201,7 +206,7 @@ public void testAsyncMajorAndMinorCompaction() throws IOException { view.sync(); // Run Compaction testMultipleWriteSteps(view, Collections.singletonList("15"), false, "15", 2, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "16"))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "16"))); // Run 2 more ingest instantsToFiles.putAll(testMultipleWriteSteps(view, Arrays.asList("17", "18"), true, "15", 2)); @@ -228,14 +233,14 @@ public void testAsyncMajorAndMinorCompaction() throws IOException { // Finish Log Compaction instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "15", 1, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "24")))); // Schedule Compaction again scheduleCompaction(view, "25"); // Finish Compaction instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("25"), false, "25", 2, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "25")))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "25")))); view.close(); } @@ -248,14 +253,14 @@ public void testIngestion() throws IOException { String firstEmptyInstantTs = "11"; HoodieCommitMetadata metadata = new HoodieCommitMetadata(); metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs)); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs), - serializeCommitMetadata(metadata)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata)); view.sync(); assertTrue(view.getLastInstant().isPresent()); - assertEquals("11", view.getLastInstant().get().getTimestamp()); + assertEquals("11", view.getLastInstant().get().requestedTime()); assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction()); PARTITIONS.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count())); @@ -291,14 +296,14 @@ public void testReplaceCommits() throws IOException { String firstEmptyInstantTs = "11"; HoodieCommitMetadata metadata = new HoodieCommitMetadata(); metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs)); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs), - serializeCommitMetadata(metadata)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs), + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata)); view.sync(); assertTrue(view.getLastInstant().isPresent()); - assertEquals("11", view.getLastInstant().get().getTimestamp()); + assertEquals("11", view.getLastInstant().get().requestedTime()); assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction()); PARTITIONS.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count())); @@ -381,12 +386,12 @@ private Map> pickFilesToReplace(Map> i } private HoodieInstant getHoodieReplaceInstant(String timestamp) { - return new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp); + return INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp); } private HoodieInstant getHoodieCommitInstant(String timestamp, boolean isDeltaCommit) { String action = isDeltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION; - return new HoodieInstant(false, action, timestamp); + return INSTANT_GENERATOR.createNewInstant(State.COMPLETED, action, timestamp); } /** @@ -406,7 +411,7 @@ public void testMultipleTransitions() throws IOException { instantsToFiles = testMultipleWriteSteps(view1, Collections.singletonList("11"), true, "11"); SyncableFileSystemView view2 = getFileSystemView(createMetaClient( - metaClient.getStorageConf().newInstance(), metaClient.getBasePath())); + metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion())); // Run 2 more ingestion on MOR table. View1 is not yet synced but View2 is instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("12", "13"), true, "11")); @@ -416,7 +421,7 @@ public void testMultipleTransitions() throws IOException { view2.sync(); SyncableFileSystemView view3 = getFileSystemView(createMetaClient( - metaClient.getStorageConf().newInstance(), metaClient.getBasePath())); + metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion())); view3.sync(); areViewsConsistent(view1, view2, PARTITIONS.size() * FILE_IDS_PER_PARTITION.size()); @@ -428,7 +433,7 @@ public void testMultipleTransitions() throws IOException { view1.sync(); areViewsConsistent(view1, view2, PARTITIONS.size() * FILE_IDS_PER_PARTITION.size()); SyncableFileSystemView view4 = getFileSystemView(createMetaClient( - metaClient.getStorageConf().newInstance(), metaClient.getBasePath())); + metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion())); view4.sync(); /* @@ -438,11 +443,11 @@ public void testMultipleTransitions() throws IOException { instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("17", "18"), true, "16", 2)); // Compaction testMultipleWriteSteps(view2, Collections.singletonList("16"), false, "16", 2, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18"))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "18"))); view1.sync(); areViewsConsistent(view1, view2, PARTITIONS.size() * FILE_IDS_PER_PARTITION.size() * 2); SyncableFileSystemView view5 = getFileSystemView(createMetaClient( - metaClient.getStorageConf().newInstance(), metaClient.getBasePath())); + metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion())); view5.sync(); /* @@ -459,13 +464,13 @@ public void testMultipleTransitions() throws IOException { instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2)); // Compaction testMultipleWriteSteps(view2, Collections.singletonList("20"), false, "20", 2, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22"))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "22"))); // Run one more round of ingestion instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("23", "24"), true, "20", 2)); view1.sync(); areViewsConsistent(view1, view2, PARTITIONS.size() * FILE_IDS_PER_PARTITION.size() * 2); SyncableFileSystemView view6 = getFileSystemView(createMetaClient( - metaClient.getStorageConf().newInstance(), metaClient.getBasePath())); + metaClient.getStorageConf().newInstance(), metaClient.getBasePath(), metaClient.getTableConfig().getTableVersion())); view6.sync(); /* @@ -478,7 +483,7 @@ public void testMultipleTransitions() throws IOException { instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3)); // Compaction testMultipleWriteSteps(view2, Collections.singletonList("28"), false, "28", 3, - Collections.singletonList(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29"))); + Collections.singletonList(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "29"))); Arrays.asList(view1, view2, view3, view4, view5, view6).forEach(v -> { v.sync(); @@ -537,7 +542,7 @@ private void testCleans(SyncableFileSystemView view, List newCleanerInst final long expTotalFileSlicesPerPartition = exp; view.sync(); assertTrue(view.getLastInstant().isPresent()); - assertEquals(newCleanerInstants.get(idx), view.getLastInstant().get().getTimestamp()); + assertEquals(newCleanerInstants.get(idx), view.getLastInstant().get().requestedTime()); assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); assertEquals(HoodieTimeline.CLEAN_ACTION, view.getLastInstant().get().getAction()); PARTITIONS.forEach(p -> { @@ -585,19 +590,19 @@ private void testRestore(SyncableFileSystemView view, List newRestoreIns HoodieInstant instant = rolledBackInstants.get(idx); try { boolean isDeltaCommit = HoodieTimeline.DELTA_COMMIT_ACTION.equalsIgnoreCase(instant.getAction()); - performRestore(instant, instantsToFiles.getOrDefault(instant.getTimestamp(), Collections.emptyList()), newRestoreInstants.get(idx), isRestore); + performRestore(instant, instantsToFiles.getOrDefault(instant.requestedTime(), Collections.emptyList()), newRestoreInstants.get(idx), isRestore); final long expTotalFileSlicesPerPartition = isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * (FILE_IDS_PER_PARTITION.size() - totalReplacedFileSlicesPerPartition)); view.sync(); assertTrue(view.getLastInstant().isPresent()); LOG.info("Last Instant is :" + view.getLastInstant().get()); if (isRestore) { - assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().getTimestamp()); + assertEquals(newRestoreInstants.get(idx), view.getLastInstant().get().requestedTime()); assertEquals(HoodieTimeline.RESTORE_ACTION, view.getLastInstant().get().getAction()); } assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); - if (HoodieTimeline.compareTimestamps(newRestoreInstants.get(idx), HoodieTimeline.GREATER_THAN_OR_EQUALS, emptyRestoreInstant + if (InstantComparison.compareTimestamps(newRestoreInstants.get(idx), GREATER_THAN_OR_EQUALS, emptyRestoreInstant )) { PARTITIONS.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count())); } else { @@ -631,7 +636,7 @@ private void performClean(String instant, List files, String cleanInstan instant.length() < 3 ? String.valueOf(Integer.parseInt(instant) + 1) : HoodieInstantTimeGenerator.instantTimePlusMillis(instant, 1), "")).collect(Collectors.toList()); - HoodieInstant cleanInflightInstant = new HoodieInstant(true, HoodieTimeline.CLEAN_ACTION, cleanInstant); + HoodieInstant cleanInflightInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, cleanInstant); metaClient.getActiveTimeline().createNewInstant(cleanInflightInstant); HoodieCleanMetadata cleanMetadata = CleanerUtils.convertCleanMetadata(cleanInstant, Option.empty(), cleanStats, Collections.EMPTY_MAP); metaClient.getActiveTimeline().saveAsComplete(cleanInflightInstant, @@ -665,21 +670,21 @@ private void performRestore(HoodieInstant instant, List files, String ro Collections.singletonMap(rollbackInstant, rollbackM)); HoodieInstant restoreInstant = - new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.RESTORE_ACTION, rollbackInstant); metaClient.getActiveTimeline().createNewInstant(restoreInstant); metaClient.getActiveTimeline() .saveAsComplete(restoreInstant, TimelineMetadataUtils.serializeRestoreMetadata(metadata)); } else { metaClient.getActiveTimeline().createNewInstant( - new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant)); + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant)); metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant), + INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant), TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata)); } StoragePath instantPath = HoodieTestUtils .getCompleteInstantPath(metaClient.getStorage(), metaClient.getMetaPath(), - instant.getTimestamp(), instant.getAction()); + instant.requestedTime(), instant.getAction()); boolean deleted = metaClient.getStorage().deleteFile(instantPath); assertTrue(deleted); } @@ -713,7 +718,7 @@ private void scheduleCompaction(SyncableFileSystemView view, String instantTime .flatMap(p -> view.getLatestFileSlices(p).map(s -> Pair.of(p, s))).collect(Collectors.toList()); long initialExpTotalFileSlices = PARTITIONS.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum(); - HoodieInstant compactionRequestedInstant = new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, instantTime); + HoodieInstant compactionRequestedInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, COMPACTION_ACTION, instantTime); HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty()); metaClient.getActiveTimeline().saveToCompactionRequested(compactionRequestedInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); @@ -726,7 +731,7 @@ private void scheduleCompaction(SyncableFileSystemView view, String instantTime assertFalse(fs.getBaseFile().isPresent()); }); view.getLatestMergedFileSlicesBeforeOrOn(p, instantTime).forEach(fs -> { - assertTrue(HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.GREATER_THAN, fs.getBaseInstantTime())); + assertTrue(InstantComparison.compareTimestamps(instantTime, GREATER_THAN, fs.getBaseInstantTime())); assertEquals(p, fs.getPartitionPath()); }); }); @@ -748,7 +753,7 @@ private void scheduleLogCompaction(SyncableFileSystemView view, String instantTi .flatMap(p -> view.getLatestFileSlices(p).map(s -> Pair.of(p, s))).collect(Collectors.toList()); long initialExpTotalFileSlices = PARTITIONS.stream().mapToLong(p -> view.getAllFileSlices(p).count()).sum(); - HoodieInstant logCompactionRequestedInstant = new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, instantTime); + HoodieInstant logCompactionRequestedInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, LOG_COMPACTION_ACTION, instantTime); HoodieCompactionPlan plan = CompactionUtils.buildFromFileSlices(slices, Option.empty(), Option.empty()); metaClient.getActiveTimeline().saveToLogCompactionRequested(logCompactionRequestedInstant, TimelineMetadataUtils.serializeCompactionPlan(plan)); @@ -778,14 +783,14 @@ private void scheduleLogCompaction(SyncableFileSystemView view, String instantTi private void unscheduleCompaction(SyncableFileSystemView view, String compactionInstantTime, String newLastInstant, String newBaseInstant) throws IOException { HoodieInstant instant = - new HoodieInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstantTime); + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, COMPACTION_ACTION, compactionInstantTime); boolean deleted = metaClient.getStorage().deleteFile( - new StoragePath(metaClient.getMetaPath(), instant.getFileName())); + new StoragePath(metaClient.getMetaPath(), INSTANT_FILE_NAME_GENERATOR.getFileName(instant))); ValidationUtils.checkArgument(deleted, "Unable to delete compaction instant."); view.sync(); - assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp()); + assertEquals(newLastInstant, view.getLastInstant().get().requestedTime()); PARTITIONS.forEach(p -> view.getLatestFileSlices(p) .forEach(fs -> assertEquals(newBaseInstant, fs.getBaseInstantTime()))); } @@ -801,14 +806,14 @@ private void unscheduleCompaction(SyncableFileSystemView view, String compaction private void unscheduleLogCompaction(SyncableFileSystemView view, String logCompactionInstantTime, String newLastInstant, String newBaseInstant) throws IOException { HoodieInstant instant = - new HoodieInstant(State.REQUESTED, LOG_COMPACTION_ACTION, logCompactionInstantTime); + INSTANT_GENERATOR.createNewInstant(State.REQUESTED, LOG_COMPACTION_ACTION, logCompactionInstantTime); boolean deleted = metaClient.getStorage().deleteFile( - new StoragePath(metaClient.getMetaPath(), instant.getFileName())); + new StoragePath(metaClient.getMetaPath(), INSTANT_FILE_NAME_GENERATOR.getFileName(instant))); ValidationUtils.checkArgument(deleted, "Unable to delete log compaction instant."); view.sync(); - assertEquals(newLastInstant, view.getLastInstant().get().getTimestamp()); + assertEquals(newLastInstant, view.getLastInstant().get().requestedTime()); PARTITIONS.forEach(p -> view.getLatestFileSlices(p) .forEach(fs -> assertEquals(newBaseInstant, fs.getBaseInstantTime()))); } @@ -841,7 +846,7 @@ private Map> testMultipleWriteSteps(SyncableFileSystemView boolean deltaCommit, String baseInstantForDeltaCommit, int begin) throws IOException { return testMultipleWriteSteps(view, instants, deltaCommit, baseInstantForDeltaCommit, begin, instants.stream() - .map(i -> new HoodieInstant(State.COMPLETED, + .map(i -> INSTANT_GENERATOR.createNewInstant(State.COMPLETED, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, i)) .collect(Collectors.toList())); } @@ -883,7 +888,7 @@ private Map> testMultipleWriteSteps(SyncableFileSystemView List filePaths = addInstant(metaClient, instant, deltaCommit); view.sync(); assertTrue(view.getLastInstant().isPresent()); - assertEquals(lastInstant.getTimestamp(), view.getLastInstant().get().getTimestamp()); + assertEquals(lastInstant.requestedTime(), view.getLastInstant().get().requestedTime()); assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); assertEquals(lastInstant.getAction(), view.getLastInstant().get().getAction(), "Expected Last=" + lastInstant + ", Found Instants=" @@ -1000,10 +1005,10 @@ private List addInstant(HoodieTableMetaClient metaClient, String instant List> writeStats = generateDataForInstant(instant, deltaCommit); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue())); - HoodieInstant inflightInstant = new HoodieInstant(true, + HoodieInstant inflightInstant = INSTANT_GENERATOR.createNewInstant(State.INFLIGHT, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, instant); metaClient.getActiveTimeline().createNewInstant(inflightInstant); - metaClient.getActiveTimeline().saveAsComplete(inflightInstant, serializeCommitMetadata(metadata)); + metaClient.getActiveTimeline().saveAsComplete(inflightInstant, serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), metadata)); return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList()); } @@ -1011,7 +1016,7 @@ private List addReplaceInstant(HoodieTableMetaClient metaClient, String List> writeStats, Map> partitionToReplaceFileIds) throws IOException { // created requested - HoodieInstant newRequestedInstant = new HoodieInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instant); + HoodieInstant newRequestedInstant = INSTANT_GENERATOR.createNewInstant(State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instant); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setOperationType(WriteOperationType.UNKNOWN.name()).build(); metaClient.getActiveTimeline().saveToPendingReplaceCommit(newRequestedInstant, @@ -1026,7 +1031,7 @@ private List addReplaceInstant(HoodieTableMetaClient metaClient, String replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds); metaClient.getActiveTimeline().saveAsComplete( inflightInstant, - serializeCommitMetadata(replaceCommitMetadata)); + serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), replaceCommitMetadata)); return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList()); } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java index c47276885460..28a157d5a0b4 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieCommonTestHarness.java @@ -258,11 +258,11 @@ protected Boolean hasPendingCommits() { Set completedInstants = completedTimeline .getInstants() .stream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toSet()); + .map(HoodieInstant::requestedTime).collect(Collectors.toSet()); List pendingInstants = timeline .getInstants() .stream() - .map(HoodieInstant::getTimestamp) + .map(HoodieInstant::requestedTime) .filter(t -> !completedInstants.contains(t)) .collect(Collectors.toList()); return !pendingInstants.isEmpty(); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index 28b7dbfbfcdb..54f1363fca9a 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -53,10 +53,12 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.log.TestLogReaderUtils; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantGenerator; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.FileIOUtils; @@ -133,6 +135,7 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.deleteSavepointCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.logFileName; import static org.apache.hudi.common.testutils.HoodieCommonTestHarness.BASE_FILE_EXTENSION; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; import static org.apache.hudi.common.util.CommitUtils.buildMetadata; import static org.apache.hudi.common.util.CommitUtils.getCommitActionType; @@ -159,6 +162,7 @@ public class HoodieTestTable implements AutoCloseable { protected String currentInstantTime; private boolean isNonPartitioned = false; protected Option context; + protected final InstantGenerator instantGenerator = new DefaultInstantGenerator(); protected HoodieTestTable(String basePath, HoodieStorage storage, HoodieTableMetaClient metaClient) { @@ -200,7 +204,7 @@ public static String makeNewCommitTime() { } public static String makeNewCommitTime(Instant dateTime) { - return HoodieActiveTimeline.formatDate(Date.from(dateTime)); + return TimelineUtils.formatDate(Date.from(dateTime)); } public static List makeIncrementalCommitTimes(int num, int firstOffsetSeconds, int deltaSecs) { @@ -243,7 +247,7 @@ public HoodieTestTable addCommit(String instantTime, Option completionTime, Option metadata) throws Exception { createRequestedCommit(basePath, instantTime); createInflightCommit(basePath, instantTime); - createCommit(basePath, instantTime, completionTime, metadata); + createCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, completionTime, metadata); currentInstantTime = instantTime; return this; } @@ -289,9 +293,9 @@ public HoodieCommitMetadata createCommitMetadata(WriteOperationType operationTyp public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { if (metaClient.getTableType() == HoodieTableType.COPY_ON_WRITE) { - createCommit(basePath, instantTime, Option.of(metadata)); + createCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, Option.of(metadata)); } else { - createDeltaCommit(basePath, instantTime, metadata); + createDeltaCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, metadata); } inflightCommits.remove(instantTime); currentInstantTime = instantTime; @@ -317,7 +321,7 @@ public HoodieTestTable addDeltaCommit(String instantTime) throws Exception { public HoodieTestTable addDeltaCommit(String instantTime, HoodieCommitMetadata metadata) throws Exception { createRequestedDeltaCommit(basePath, instantTime); createInflightDeltaCommit(basePath, instantTime); - createDeltaCommit(basePath, instantTime, metadata); + createDeltaCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, metadata); currentInstantTime = instantTime; return this; } @@ -328,22 +332,22 @@ public HoodieTestTable addReplaceCommit( Option inflightReplaceMetadata, HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception { createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata); - createInflightReplaceCommit(basePath, instantTime, inflightReplaceMetadata); - createReplaceCommit(basePath, instantTime, completeReplaceMetadata); + createInflightReplaceCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); + createReplaceCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, completeReplaceMetadata); currentInstantTime = instantTime; return this; } public HoodieTestTable addPendingReplace(String instantTime, Option requestedReplaceMetadata, Option inflightReplaceMetadata) throws Exception { createRequestedReplaceCommit(basePath, instantTime, requestedReplaceMetadata); - createInflightReplaceCommit(basePath, instantTime, inflightReplaceMetadata); + createInflightReplaceCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); currentInstantTime = instantTime; return this; } public HoodieTestTable addPendingCluster(String instantTime, HoodieRequestedReplaceMetadata requestedReplaceMetadata, Option inflightReplaceMetadata) throws Exception { createRequestedClusterCommit(basePath, instantTime, requestedReplaceMetadata); - createInflightClusterCommit(basePath, instantTime, inflightReplaceMetadata); + createInflightClusterCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); currentInstantTime = instantTime; return this; } @@ -355,7 +359,7 @@ public HoodieTestTable addRequestedCluster(String instantTime, HoodieRequestedRe } public HoodieTestTable addInflightCluster(String instantTime, Option inflightReplaceMetadata) throws Exception { - createInflightClusterCommit(basePath, instantTime, inflightReplaceMetadata); + createInflightClusterCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); currentInstantTime = instantTime; return this; } @@ -366,8 +370,8 @@ public HoodieTestTable addCluster( Option inflightReplaceMetadata, HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception { createRequestedClusterCommit(basePath, instantTime, requestedReplaceMetadata); - createInflightClusterCommit(basePath, instantTime, inflightReplaceMetadata); - createReplaceCommit(basePath, instantTime, completeReplaceMetadata); + createInflightClusterCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); + createReplaceCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, completeReplaceMetadata); currentInstantTime = instantTime; return this; } @@ -379,7 +383,7 @@ public HoodieTestTable addRequestedReplace(String instantTime, Option inflightReplaceMetadata) throws Exception { - createInflightReplaceCommit(basePath, instantTime, inflightReplaceMetadata); + createInflightReplaceCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, inflightReplaceMetadata); currentInstantTime = instantTime; return this; } @@ -537,7 +541,7 @@ public HoodieTestTable addRequestedCompaction(String instantTime) throws IOExcep HoodieCompactionPlan compactionPlan = CompactionUtils .buildFromFileSlices(fileSlices.stream().map(fs -> Pair.of(fs.getPartitionPath(), fs)) .collect(Collectors.toList()), Option.empty(), Option.empty()); - HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); + HoodieInstant compactionInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); currentInstantTime = instantTime; @@ -545,7 +549,7 @@ public HoodieTestTable addRequestedCompaction(String instantTime) throws IOExcep } public HoodieTestTable addRequestedCompaction(String instantTime, HoodieCompactionPlan compactionPlan) throws IOException { - HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); + HoodieInstant compactionInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); currentInstantTime = instantTime; @@ -576,7 +580,7 @@ public HoodieTestTable addInflightCompaction(String instantTime, HoodieCommitMet public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception { addInflightCompaction(instantTime, commitMetadata); this.inflightCommits.remove(instantTime); - createCommit(basePath, instantTime, Option.of(commitMetadata)); + createCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, Option.of(commitMetadata)); return this; } @@ -622,7 +626,7 @@ private Pair genera } public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException { - createCommit(basePath, instantTime, Option.of(metadata)); + createCommit(COMMIT_METADATA_SER_DE, basePath, instantTime, Option.of(metadata)); inflightCommits.remove(instantTime); currentInstantTime = instantTime; return this; @@ -1012,11 +1016,11 @@ public HoodieTestTable doRestore(String commitToRestoreTo, String restoreTime) t for (HoodieInstant commitInstantToRollback : commitsToRollback) { Option commitMetadata = getCommitMeta(commitInstantToRollback); if (!commitMetadata.isPresent()) { - throw new IllegalArgumentException("Instant to rollback not present in timeline: " + commitInstantToRollback.getTimestamp()); + throw new IllegalArgumentException("Instant to rollback not present in timeline: " + commitInstantToRollback.requestedTime()); } Map> partitionFiles = getPartitionFiles(commitMetadata.get()); - rollbackMetadataMap.put(commitInstantToRollback.getTimestamp(), - Collections.singletonList(getRollbackMetadata(commitInstantToRollback.getTimestamp(), partitionFiles, false))); + rollbackMetadataMap.put(commitInstantToRollback.requestedTime(), + Collections.singletonList(getRollbackMetadata(commitInstantToRollback.requestedTime(), partitionFiles, false))); for (Map.Entry> entry : partitionFiles.entrySet()) { deleteFilesInPartition(entry.getKey(), entry.getValue()); } @@ -1228,7 +1232,7 @@ public HoodieCommitMetadata doWriteOperation(String commitTime, WriteOperationTy private Option getMetadataForInstant(String instantTime) { metaClient = HoodieTableMetaClient.reload(metaClient); Option hoodieInstant = metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().filter(i -> i.getTimestamp().equals(instantTime)).firstInstant(); + .filterCompletedInstants().filter(i -> i.requestedTime().equals(instantTime)).firstInstant(); try { if (hoodieInstant.isPresent()) { return getCommitMeta(hoodieInstant.get()); @@ -1249,8 +1253,9 @@ private Option getCommitMeta(HoodieInstant hoodieInstant) return Option.of(replaceCommitMetadata); case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.COMMIT_ACTION: - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + hoodieInstant, + metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); return Option.of(commitMetadata); default: throw new IllegalArgumentException("Unknown instant action" + hoodieInstant.getAction()); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java index 492240138b56..704c97151771 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestClusteringUtils.java @@ -49,6 +49,7 @@ import java.util.UUID; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -93,7 +94,7 @@ public void testClusteringPlanMultipleInstants() throws Exception { createRequestedReplaceInstantNotClustering("3"); // create replace.requested without any metadata content. This instant should be ignored by ClusteringUtils - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4")); + metaClient.getActiveTimeline().createNewInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, "4")); metaClient.reloadActiveTimeline(); assertEquals(2, metaClient.getActiveTimeline().filterPendingClusteringTimeline().countInstants()); @@ -106,19 +107,19 @@ public void testClusteringPlanMultipleInstants() throws Exception { validateClusteringInstant(fileIds3, partitionPath1, clusterTime, fileGroupToInstantMap); Option lastPendingClustering = metaClient.getActiveTimeline().getLastPendingClusterInstant(); assertTrue(lastPendingClustering.isPresent()); - assertEquals("2", lastPendingClustering.get().getTimestamp()); + assertEquals("2", lastPendingClustering.get().requestedTime()); //check that it still gets picked if it is inflight HoodieInstant inflight = metaClient.getActiveTimeline().transitionClusterRequestedToInflight(lastPendingClustering.get(), Option.empty()); assertEquals(HoodieInstant.State.INFLIGHT, inflight.getState()); lastPendingClustering = metaClient.reloadActiveTimeline().getLastPendingClusterInstant(); - assertEquals("2", lastPendingClustering.get().getTimestamp()); + assertEquals("2", lastPendingClustering.get().requestedTime()); //now that it is complete, the first instant should be picked HoodieInstant complete = metaClient.getActiveTimeline().transitionClusterInflightToComplete(false, inflight, Option.empty()); assertEquals(HoodieInstant.State.COMPLETED, complete.getState()); lastPendingClustering = metaClient.reloadActiveTimeline().getLastPendingClusterInstant(); - assertEquals("1", lastPendingClustering.get().getTimestamp()); + assertEquals("1", lastPendingClustering.get().requestedTime()); } // replacecommit.inflight doesn't have clustering plan. @@ -133,9 +134,9 @@ public void testClusteringPlanInflight() throws Exception { String clusterTime1 = "1"; HoodieInstant requestedInstant = createRequestedClusterInstant(partitionPath1, clusterTime1, fileIds1); HoodieInstant inflightInstant = metaClient.getActiveTimeline().transitionReplaceRequestedToInflight(requestedInstant, Option.empty()); - assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), requestedInstant)); + assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), requestedInstant, INSTANT_GENERATOR)); HoodieClusteringPlan requestedClusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, requestedInstant).get().getRight(); - assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), inflightInstant)); + assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline(), inflightInstant, INSTANT_GENERATOR)); HoodieClusteringPlan inflightClusteringPlan = ClusteringUtils.getClusteringPlan(metaClient, inflightInstant).get().getRight(); assertEquals(requestedClusteringPlan, inflightClusteringPlan); } @@ -167,14 +168,14 @@ public void testGetOldestInstantToRetainForClustering() throws IOException { metaClient.reloadActiveTimeline(); Option actual = ClusteringUtils.getEarliestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient, null); assertTrue(actual.isPresent()); - assertEquals(clusterTime1, actual.get().getTimestamp(), "no clean in timeline, retain first replace commit"); + assertEquals(clusterTime1, actual.get().requestedTime(), "no clean in timeline, retain first replace commit"); String cleanTime1 = "4"; - HoodieInstant requestedInstant4 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); + HoodieInstant requestedInstant4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); HoodieCleanerPlan cleanerPlan1 = HoodieCleanerPlan.newBuilder() .setEarliestInstantToRetainBuilder(HoodieActionInstant.newBuilder() .setAction(completedInstant3.getAction()) - .setTimestamp(completedInstant3.getTimestamp()) + .setTimestamp(completedInstant3.requestedTime()) .setState(completedInstant3.getState().name())) .setPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS.name()) .setFilesToBeDeletedPerPartition(new HashMap<>()) @@ -183,12 +184,12 @@ public void testGetOldestInstantToRetainForClustering() throws IOException { metaClient.getActiveTimeline().saveToCleanRequested(requestedInstant4, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan1)); HoodieInstant inflightInstant4 = metaClient.getActiveTimeline().transitionCleanRequestedToInflight(requestedInstant4, Option.empty()); HoodieCleanMetadata cleanMetadata = new HoodieCleanMetadata(cleanTime1, 1L, 1, - completedInstant3.getTimestamp(), "", Collections.emptyMap(), 0, Collections.emptyMap(), Collections.emptyMap()); + completedInstant3.requestedTime(), "", Collections.emptyMap(), 0, Collections.emptyMap(), Collections.emptyMap()); metaClient.getActiveTimeline().transitionCleanInflightToComplete(true, inflightInstant4, TimelineMetadataUtils.serializeCleanMetadata(cleanMetadata)); metaClient.reloadActiveTimeline(); actual = ClusteringUtils.getEarliestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient, null); - assertEquals(clusterTime3, actual.get().getTimestamp(), + assertEquals(clusterTime3, actual.get().requestedTime(), "retain the first replace commit after the earliestInstantToRetain "); } @@ -204,7 +205,7 @@ public void testGetOldestInstantToRetainForClusteringKeepFileVersion() throws IO metaClient.getActiveTimeline().transitionClusterInflightToComplete(true, inflightInstant1, Option.empty()); String cleanTime1 = "2"; - HoodieInstant requestedInstant2 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); + HoodieInstant requestedInstant2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLEAN_ACTION, cleanTime1); HoodieCleanerPlan cleanerPlan1 = new HoodieCleanerPlan(null, clusterTime1, HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS.name(), Collections.emptyMap(), CleanPlanV2MigrationHandler.VERSION, Collections.emptyMap(), Collections.emptyList(), Collections.EMPTY_MAP); @@ -226,19 +227,19 @@ public void testGetOldestInstantToRetainForClusteringKeepFileVersion() throws IO metaClient.reloadActiveTimeline(); Option actual = ClusteringUtils.getEarliestInstantToRetainForClustering(metaClient.getActiveTimeline(), metaClient, HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS); - assertEquals(clusterTime2, actual.get().getTimestamp(), + assertEquals(clusterTime2, actual.get().requestedTime(), "retain the first replace commit after the last complete clean "); } private void validateClusteringInstant(List fileIds, String partitionPath, String expectedInstantTime, Map fileGroupToInstantMap) { for (String fileId : fileIds) { - assertEquals(expectedInstantTime, fileGroupToInstantMap.get(new HoodieFileGroupId(partitionPath, fileId)).getTimestamp()); + assertEquals(expectedInstantTime, fileGroupToInstantMap.get(new HoodieFileGroupId(partitionPath, fileId)).requestedTime()); } } private HoodieInstant createRequestedReplaceInstantNotClustering(String instantTime) throws IOException { - HoodieInstant newRequestedInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); + HoodieInstant newRequestedInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setOperationType(WriteOperationType.UNKNOWN.name()).build(); metaClient.getActiveTimeline().saveToPendingReplaceCommit(newRequestedInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); @@ -254,7 +255,7 @@ private HoodieInstant createRequestedClusterInstant(String partitionPath1, Strin HoodieClusteringPlan clusteringPlan = ClusteringUtils.createClusteringPlan(CLUSTERING_STRATEGY_CLASS, STRATEGY_PARAMS, fileSliceGroups, Collections.emptyMap()); - HoodieInstant clusteringInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusterTime); + HoodieInstant clusteringInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusterTime); HoodieRequestedReplaceMetadata requestedReplaceMetadata = HoodieRequestedReplaceMetadata.newBuilder() .setClusteringPlan(clusteringPlan).setOperationType(WriteOperationType.CLUSTER.name()).build(); metaClient.getActiveTimeline().saveToPendingClusterCommit(clusteringInstant, TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata)); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java index 7421d9e0c2c3..bc866640d5a3 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java @@ -48,6 +48,8 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMPACTION_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.REPLACE_COMMIT_ACTION; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.util.CommitUtils.getCheckpointValueAsString; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -125,7 +127,7 @@ public void testReplaceMetadataCreation() { @Test public void testGetValidCheckpointForCurrentWriter() throws IOException { init(); - HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient); + HoodieActiveTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); // Deltacommit 1 completed: (id1, 3) addDeltaCommit(timeline, "20230913001000000", ID1, "3", true); @@ -161,32 +163,32 @@ private HoodieWriteStat createWriteStat(String partition, String fileId) { private void addDeltaCommit(HoodieActiveTimeline timeline, String ts, String id, String batchId, boolean isCompleted) throws IOException { - HoodieInstant instant = new HoodieInstant( + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.REQUESTED, HoodieTimeline.DELTA_COMMIT_ACTION, ts); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); commitMetadata.setOperationType(WriteOperationType.UPSERT); commitMetadata.addMetadata(SINK_CHECKPOINT_KEY, getCheckpointValueAsString(id, batchId)); timeline.createNewInstant(instant); - timeline.transitionRequestedToInflight(instant, TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + timeline.transitionRequestedToInflight(instant, TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); if (isCompleted) { timeline.saveAsComplete( - new HoodieInstant(true, instant.getAction(), instant.getTimestamp()), - TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instant.getAction(), instant.requestedTime()), + TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); } } private void addCommit(HoodieActiveTimeline timeline, String ts) throws IOException { - HoodieInstant instant = new HoodieInstant( + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant( HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, ts); HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); commitMetadata.setOperationType(WriteOperationType.COMPACT); timeline.createNewInstant(instant); - timeline.transitionRequestedToInflight(instant, TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + timeline.transitionRequestedToInflight(instant, TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); timeline.saveAsComplete( - new HoodieInstant(true, instant.getAction(), instant.getTimestamp()), - TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instant.getAction(), instant.requestedTime()), + TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); } private void addRequestedCompaction(HoodieActiveTimeline timeline, @@ -198,7 +200,7 @@ private void addRequestedCompaction(HoodieActiveTimeline timeline, .setPreserveHoodieMetadata(true) .build(); timeline.saveToCompactionRequested( - new HoodieInstant(HoodieInstant.State.REQUESTED, COMPACTION_ACTION, ts), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, COMPACTION_ACTION, ts), TimelineMetadataUtils.serializeCompactionPlan(compactionPlan) ); } @@ -212,7 +214,7 @@ private void addRequestedReplaceCommit(HoodieActiveTimeline timeline, .setClusteringPlan(new HoodieClusteringPlan()) .build(); timeline.saveToPendingClusterCommit( - new HoodieInstant(HoodieInstant.State.REQUESTED, CLUSTERING_ACTION, ts), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, CLUSTERING_ACTION, ts), TimelineMetadataUtils.serializeRequestedReplaceMetadata(requestedReplaceMetadata) ); } diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java index 715abe789e2f..4a7b811fe8f0 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/common/util/TestCompactionUtils.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.versioning.compaction.CompactionPlanMigrator; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; import org.apache.hudi.common.testutils.CompactionTestUtils.DummyHoodieBaseFile; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.util.collection.Pair; @@ -56,6 +57,7 @@ import static org.apache.hudi.common.testutils.CompactionTestUtils.scheduleCompaction; import static org.apache.hudi.common.testutils.CompactionTestUtils.setupAndValidateCompactionOperations; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createMetaClient; import static org.apache.hudi.common.util.CompactionUtils.COMPACTION_METADATA_VERSION_1; import static org.apache.hudi.common.util.CompactionUtils.LATEST_COMPACTION_METADATA_VERSION; @@ -261,29 +263,29 @@ public void testGetDeltaCommitsSinceLatestCompaction(boolean hasCompletedCompact if (hasCompletedCompaction) { assertEquals( Stream.of( - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "900")) + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "900")) .collect(Collectors.toList()), actual.getLeft().getInstants()); assertEquals( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "600"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "600"), actual.getRight()); } else { assertEquals( Stream.of( - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "200"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "300"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "400"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "500"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "900")) + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "200"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "300"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "400"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "500"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "900")) .collect(Collectors.toList()), actual.getLeft().getInstants()); assertEquals( - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), actual.getRight()); } } @@ -292,53 +294,53 @@ public void testGetDeltaCommitsSinceLatestCompaction(boolean hasCompletedCompact public void testGetDeltaCommitsSinceLastCompactionWithCompletedReplaceCommits() { // 4th replace commit. HoodieActiveTimeline timeline = new MockHoodieActiveTimeline( - Stream.of(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "02"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "03"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "04"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "05"), - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "06"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")).collect(Collectors.toList())); + Stream.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "02"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "04"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "05"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "06"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "09")).collect(Collectors.toList())); Pair actual = CompactionUtils.getDeltaCommitsSinceLatestCompaction(timeline).get(); assertEquals( Stream.of( - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) .collect(Collectors.toList()), actual.getLeft().getInstants()); assertEquals( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "06"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "06"), actual.getRight()); // mix of compaction commit and replace commit. timeline = new MockHoodieActiveTimeline( - Stream.of(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "02"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "03"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "04"), - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "05"), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, "06"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), - new HoodieInstant(true, HoodieTimeline.DELTA_COMMIT_ACTION, "09"), - new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, "10")).collect(Collectors.toList())); + Stream.of(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "01"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "02"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "03"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "04"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, "06"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "09"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, "10")).collect(Collectors.toList())); actual = CompactionUtils.getDeltaCommitsSinceLatestCompaction(timeline).get(); 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")) + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "07"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "08"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "09")) .collect(Collectors.toList()), actual.getLeft().getInstants()); assertEquals( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "05"), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "05"), actual.getRight()); } @@ -355,16 +357,16 @@ public void testGetOldestInstantToKeepForCompaction(boolean hasCompletedCompacti Option actual = CompactionUtils.getEarliestInstantToRetainForCompaction(timeline, 20); if (hasCompletedCompaction) { - assertEquals(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "600"), actual.get()); + assertEquals(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "600"), actual.get()); } else { - assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), actual.get()); + assertEquals(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "100"), actual.get()); } actual = CompactionUtils.getEarliestInstantToRetainForCompaction(timeline, 3); - assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), actual.get()); + assertEquals(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "700"), actual.get()); actual = CompactionUtils.getEarliestInstantToRetainForCompaction(timeline, 2); - assertEquals(new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), actual.get()); + assertEquals(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "800"), actual.get()); } @Test @@ -375,17 +377,17 @@ public void testGetOldestInstantToKeepForCompactionWithEmptyDeltaCommits() { private HoodieActiveTimeline prepareTimeline(boolean hasCompletedCompaction) { List instants = new ArrayList<>(); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "100", "110")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "200", "210")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "300", "310")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "400", "410")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "500", "510")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "100", "110")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "200", "210")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "300", "310")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "400", "410")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "500", "510")); if (hasCompletedCompaction) { - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "600", "610")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "600", "610")); } - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "700", "710")); - instants.add(new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "800", "810")); - instants.add(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "900", "910")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "700", "710")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "800", "810")); + instants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "900", "910")); return new MockHoodieActiveTimeline(instants); } @@ -430,7 +432,7 @@ protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; } - class MockHoodieActiveTimeline extends HoodieActiveTimeline { + class MockHoodieActiveTimeline extends ActiveTimelineV2 { public MockHoodieActiveTimeline() { super(); diff --git a/hudi-hadoop-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java b/hudi-hadoop-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java index 4fbf300356c1..fe53e575fd80 100644 --- a/hudi-hadoop-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java +++ b/hudi-hadoop-common/src/test/java/org/apache/hudi/internal/schema/io/TestFileBasedInternalSchemaStorageManager.java @@ -35,6 +35,8 @@ import java.io.IOException; import java.util.Arrays; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -56,7 +58,7 @@ public void tearDown() throws Exception { @Test public void testPersistAndReadHistorySchemaStr() throws IOException { - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); FileBasedInternalSchemaStorageManager fm = new FileBasedInternalSchemaStorageManager(metaClient); InternalSchema currentSchema = getSimpleSchema(); currentSchema.setSchemaId(0L); @@ -97,12 +99,12 @@ public void testPersistAndReadHistorySchemaStr() throws IOException { private void simulateCommit(String commitTime) { if (timeline == null) { - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); } - HoodieInstant instant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, commitTime); + HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMMIT_ACTION, commitTime); timeline.createNewInstant(instant); timeline.transitionRequestedToInflight(instant, Option.empty()); - timeline.saveAsComplete(new HoodieInstant(true, instant.getAction(), instant.getTimestamp()), + timeline.saveAsComplete(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instant.getAction(), instant.requestedTime()), Option.empty()); } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java index 90d44b79f86e..2636159fb7dc 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieCopyOnWriteTableInputFormat.java @@ -276,7 +276,7 @@ private List listStatusForSnapshotMode(JobConf job, String basePath = tableMetaClient.getBasePath().toString(); Map fsViewCache = new HashMap<>(); HoodieTimeline timeline = getActiveTimeline(tableMetaClient, shouldIncludePendingCommits); - Option queryInstant = queryCommitInstant.or(() -> timeline.lastInstant().map(HoodieInstant::getTimestamp)); + Option queryInstant = queryCommitInstant.or(() -> timeline.lastInstant().map(HoodieInstant::requestedTime)); validateInstant(timeline, queryInstant); try { diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java index f611eaa58f58..e74b48efd656 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieFileGroupReaderBasedRecordReader.java @@ -237,7 +237,7 @@ private static String getLatestCommitTime(InputSplit split, HoodieTableMetaClien } Option lastInstant = metaClient.getCommitsTimeline().lastInstant(); if (lastInstant.isPresent()) { - return lastInstant.get().getTimestamp(); + return lastInstant.get().requestedTime(); } else { return EMPTY_STRING; } diff --git a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java index cddf7fbb86a8..b59cdc5bc1f4 100644 --- a/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieHFileInputFormat.java @@ -18,7 +18,7 @@ package org.apache.hudi.hadoop; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.hadoop.utils.HoodieInputFormatUtils; import org.apache.hadoop.fs.FileSystem; @@ -38,7 +38,7 @@ @UseFileSplitsFromInputFormat public class HoodieHFileInputFormat extends HoodieCopyOnWriteTableInputFormat { - protected HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) { + protected HoodieTimeline filterInstantsTimeline(HoodieTimeline timeline) { return HoodieInputFormatUtils.filterInstantsTimeline(timeline); } 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 c24209e37b1f..4d4d8a4da0e8 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 @@ -223,7 +223,7 @@ protected List listStatusForIncrementalMode(JobConf job, } Option virtualKeyInfoOpt = getHoodieVirtualKeyInfo(tableMetaClient); - String maxCommitTime = fsView.getLastInstant().get().getTimestamp(); + String maxCommitTime = fsView.getLastInstant().get().requestedTime(); // step6 result.addAll(collectAllIncrementalFiles(fileGroups, maxCommitTime, basePath.toString(), candidateFileStatus, virtualKeyInfoOpt)); return result; @@ -372,7 +372,7 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieBaseFi HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); checkState(latestCompletedInstant.isCompleted()); - rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); + rtFileStatus.setMaxCommitTime(latestCompletedInstant.requestedTime()); } if (baseFileStatus instanceof LocatedFileStatusWithBootstrapBaseFile || baseFileStatus instanceof FileStatusWithBootstrapBaseFile) { @@ -403,7 +403,7 @@ private static RealtimeFileStatus createRealtimeFileStatusUnchecked(HoodieLogFil HoodieInstant latestCompletedInstant = latestCompletedInstantOpt.get(); checkState(latestCompletedInstant.isCompleted()); - rtFileStatus.setMaxCommitTime(latestCompletedInstant.getTimestamp()); + rtFileStatus.setMaxCommitTime(latestCompletedInstant.requestedTime()); } return rtFileStatus; 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 b51ad4626615..c3e9a8504d07 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 @@ -27,9 +27,8 @@ import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieDefaultTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView; @@ -218,16 +217,16 @@ public static FileInputFormat getInputFormat(String path, boolean realtime, Conf * @param timeline * @return */ - public static HoodieDefaultTimeline filterInstantsTimeline(HoodieDefaultTimeline timeline) { - HoodieDefaultTimeline commitsAndCompactionTimeline = timeline.getWriteTimeline(); + public static HoodieTimeline filterInstantsTimeline(HoodieTimeline timeline) { + HoodieTimeline commitsAndCompactionTimeline = (HoodieTimeline)timeline.getWriteTimeline(); Option pendingCompactionInstant = commitsAndCompactionTimeline .filterPendingCompactionTimeline().firstInstant(); if (pendingCompactionInstant.isPresent()) { - HoodieDefaultTimeline instantsTimeline = commitsAndCompactionTimeline - .findInstantsBefore(pendingCompactionInstant.get().getTimestamp()); + HoodieTimeline instantsTimeline = (HoodieTimeline)(commitsAndCompactionTimeline + .findInstantsBefore(pendingCompactionInstant.get().requestedTime())); int numCommitsFilteredByCompaction = commitsAndCompactionTimeline.getCommitsTimeline().countInstants() - instantsTimeline.getCommitsTimeline().countInstants(); - LOG.info("Earliest pending compaction instant is: " + pendingCompactionInstant.get().getTimestamp() + LOG.info("Earliest pending compaction instant is: " + pendingCompactionInstant.get().requestedTime() + " skipping " + numCommitsFilteredByCompaction + " commits"); return instantsTimeline; @@ -252,8 +251,8 @@ public static Option getAffectedPartitions(List commitsTo List inputPaths) throws IOException { Set partitionsToList = new HashSet<>(); for (HoodieInstant commit : commitsToCheck) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get(), - HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = tableMetaClient.getCommitMetadataSerDe().deserialize(commit, + timeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); partitionsToList.addAll(commitMetadata.getPartitionToWriteStats().keySet()); } if (partitionsToList.isEmpty()) { @@ -300,7 +299,7 @@ public static Option getAffectedPartitions(List commitsTo */ public static Option getFilteredCommitsTimeline(JobContext job, HoodieTableMetaClient tableMetaClient) { String tableName = tableMetaClient.getTableConfig().getTableName(); - HoodieDefaultTimeline baseTimeline; + HoodieTimeline baseTimeline; if (HoodieHiveUtils.stopAtCompaction(job, tableName)) { baseTimeline = filterInstantsTimeline(tableMetaClient.getActiveTimeline()); } else { @@ -425,7 +424,7 @@ public static List filterIncrementalFileStatus(Job job, HoodieTableM Arrays.stream(fileStatuses) .map(HadoopFSUtils::convertToStoragePathInfo) .collect(Collectors.toList())); - List commitsList = commitsToCheck.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + List commitsList = commitsToCheck.stream().map(HoodieInstant::requestedTime).collect(Collectors.toList()); List filteredFiles = roView.getLatestBaseFilesInRange(commitsList).collect(Collectors.toList()); List returns = new ArrayList<>(); for (HoodieBaseFile filteredFile : filteredFiles) { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java index f895aa337764..81dc1e1f93c6 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieHFileInputFormat.java @@ -54,6 +54,10 @@ import java.util.List; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -92,12 +96,12 @@ public void setUp() { public void testPendingCompactionWithActiveCommits() throws IOException { // setup 4 sample instants in timeline List instants = new ArrayList<>(); - HoodieInstant t1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant t2 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); - HoodieInstant t3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); - HoodieInstant t4 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "4"); - HoodieInstant t5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"); - HoodieInstant t6 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "6"); + HoodieInstant t1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant t2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); + HoodieInstant t3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); + HoodieInstant t4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "4"); + HoodieInstant t5 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"); + HoodieInstant t6 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "6"); instants.add(t1); instants.add(t2); @@ -106,7 +110,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { instants.add(t5); instants.add(t6); HoodieTableMetaClient metaClient = HoodieTestUtils.init(basePath.toString(), HoodieFileFormat.HFILE); - HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient); + HoodieActiveTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); // Verify getCommitsTimelineBeforePendingCompaction does not return instants after first compaction instant @@ -121,7 +125,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { // remove compaction instant and setup timeline again instants.remove(t3); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); filteredTimeline = inputFormat.filterInstantsTimeline(timeline); @@ -135,7 +139,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { // remove remaining compaction instant and setup timeline again instants.remove(t5); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); filteredTimeline = inputFormat.filterInstantsTimeline(timeline); @@ -344,7 +348,7 @@ private void createCommitFile(java.nio.file.Path basePath, String commitNumber, .resolve(commitNumber + "_" + InProcessTimeGenerator.createNewInstantTime() + ".commit").toFile(); file.createNewFile(); FileOutputStream fileOutputStream = new FileOutputStream(file); - fileOutputStream.write(serializeCommitMetadata(commitMetadata).get()); + fileOutputStream.write(serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); fileOutputStream.flush(); fileOutputStream.close(); } @@ -352,7 +356,7 @@ private void createCommitFile(java.nio.file.Path basePath, String commitNumber, private File createCompactionFile(java.nio.file.Path basePath, String commitTime) throws IOException { File file = basePath.resolve(".hoodie") - .resolve(HoodieTimeline.makeRequestedCompactionFileName(commitTime)).toFile(); + .resolve(INSTANT_FILE_NAME_GENERATOR.makeRequestedCompactionFileName(commitTime)).toFile(); assertTrue(file.createNewFile()); FileOutputStream os = new FileOutputStream(file); try { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java index 7d7a2eec626c..30331a072fae 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/TestHoodieParquetInputFormat.java @@ -80,6 +80,10 @@ import java.util.List; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.TIMELINE_FACTORY; import static org.apache.hudi.common.testutils.SchemaTestUtil.getSchemaFromResource; import static org.apache.hudi.hadoop.HoodieColumnProjectionUtils.READ_COLUMN_NAMES_CONF_STR; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -121,12 +125,12 @@ public void setUp() { public void testPendingCompactionWithActiveCommits() throws IOException { // setup 4 sample instants in timeline List instants = new ArrayList<>(); - HoodieInstant t1 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieInstant t2 = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); - HoodieInstant t3 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); - HoodieInstant t4 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "4"); - HoodieInstant t5 = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"); - HoodieInstant t6 = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "6"); + HoodieInstant t1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant t2 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "2"); + HoodieInstant t3 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "3"); + HoodieInstant t4 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "4"); + HoodieInstant t5 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "5"); + HoodieInstant t6 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "6"); instants.add(t1); instants.add(t2); @@ -135,7 +139,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { instants.add(t5); instants.add(t6); HoodieTableMetaClient metaClient = HoodieTestUtils.init(basePath.toString()); - HoodieActiveTimeline timeline = new HoodieActiveTimeline(metaClient); + HoodieActiveTimeline timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); // Verify getCommitsTimelineBeforePendingCompaction does not return instants after first compaction instant @@ -148,7 +152,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { assertFalse(filteredTimeline.containsInstant(t6)); // remove compaction instant and setup timeline again instants.remove(t3); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); filteredTimeline = HoodieInputFormatUtils.filterInstantsTimeline(timeline); @@ -162,7 +166,7 @@ public void testPendingCompactionWithActiveCommits() throws IOException { // remove remaining compaction instant and setup timeline again instants.remove(t5); - timeline = new HoodieActiveTimeline(metaClient); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); timeline.setInstants(instants); filteredTimeline = HoodieInputFormatUtils.filterInstantsTimeline(timeline); @@ -199,7 +203,7 @@ public void testInputFormatLoadForNonPartitionedAndVirtualKeyedTable() throws IO true, schema); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(basePath.toString(), "100", Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, basePath.toString(), "100", Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(jobConf, partitionDir.getPath()); @@ -500,7 +504,7 @@ private void createCommitFile(java.nio.file.Path basePath, String commitNumber, .resolve(commitNumber + "_" + InProcessTimeGenerator.createNewInstantTime() + ".commit").toFile(); file.createNewFile(); FileOutputStream fileOutputStream = new FileOutputStream(file); - fileOutputStream.write(serializeCommitMetadata(commitMetadata).get()); + fileOutputStream.write(serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); fileOutputStream.flush(); fileOutputStream.close(); } @@ -508,7 +512,7 @@ private void createCommitFile(java.nio.file.Path basePath, String commitNumber, private File createCompactionFile(java.nio.file.Path basePath, String commitTime) throws IOException { File file = basePath.resolve(".hoodie") - .resolve(HoodieTimeline.makeRequestedCompactionFileName(commitTime)).toFile(); + .resolve(INSTANT_FILE_NAME_GENERATOR.makeRequestedCompactionFileName(commitTime)).toFile(); assertTrue(file.createNewFile()); FileOutputStream os = new FileOutputStream(file); try { diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/hive/TestHoodieCombineHiveInputFormat.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/hive/TestHoodieCombineHiveInputFormat.java index e1fd5fc68743..a7ba91e39881 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/hive/TestHoodieCombineHiveInputFormat.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/hive/TestHoodieCombineHiveInputFormat.java @@ -77,6 +77,7 @@ import static org.apache.hadoop.hive.ql.exec.Utilities.HAS_MAP_WORK; import static org.apache.hadoop.hive.ql.exec.Utilities.MAPRED_MAPPER_CLASS; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -133,14 +134,14 @@ public void testInternalSchemaCacheForMR() throws Exception { // mock the latest schema to the commit metadata InternalSchema internalSchema = AvroInternalSchemaConverter.convert(schema); commitMetadataOne.addMetadata(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema)); - FileCreateUtils.createCommit(path1.toString(), commitTime, Option.of(commitMetadataOne)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, path1.toString(), commitTime, Option.of(commitMetadataOne)); // Create 3 parquet files with 10 records each for partition 2 File partitionDirTwo = InputFormatTestUtil.prepareParquetTable(path2, schema, 3, numRecords, commitTime); HoodieCommitMetadata commitMetadataTwo = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); // Mock the latest schema to the commit metadata commitMetadataTwo.addMetadata(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema)); - FileCreateUtils.createCommit(path2.toString(), commitTime, Option.of(commitMetadataTwo)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, path2.toString(), commitTime, Option.of(commitMetadataTwo)); // Enable schema evolution conf.set("hoodie.schema.on.read.enable", "true"); @@ -208,7 +209,7 @@ public void multiPartitionReadersRealtimeCombineHoodieInputFormat() throws Excep .prepareMultiPartitionedParquetTable(tempDir, schema, 3, numRecords, commitTime, HoodieTableType.MERGE_ON_READ); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, tempDir.toString(), commitTime, Option.of(commitMetadata)); TableDesc tblDesc = Utilities.defaultTd; // Set the input format @@ -290,7 +291,7 @@ public void multiLevelPartitionReadersRealtimeCombineHoodieInputFormat() throws File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, tempDir.toString(), commitTime, Option.of(commitMetadata)); TableDesc tblDesc = Utilities.defaultTd; // Set the input format @@ -373,7 +374,7 @@ public void testMultiReaderRealtimeCombineHoodieInputFormat() throws Exception { File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, tempDir.toString(), commitTime, Option.of(commitMetadata)); String newCommitTime = "101"; // to trigger the bug of HUDI-1772, only update fileid2 @@ -444,7 +445,7 @@ public void testHoodieRealtimeCombineHoodieInputFormat() throws Exception { File partitionDir = InputFormatTestUtil.prepareParquetTable(tempDir, schema, 3, numRecords, commitTime); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(tempDir.toString(), commitTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, tempDir.toString(), commitTime, Option.of(commitMetadata)); // insert 1000 update records to log file 0 String newCommitTime = "101"; diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadSnapshotReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadSnapshotReader.java index 463ad5a2ebc1..d63154311364 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadSnapshotReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieMergeOnReadSnapshotReader.java @@ -62,6 +62,7 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; import static org.apache.hudi.hadoop.fs.HadoopFSUtils.getRelativePartitionPath; import static org.apache.hudi.hadoop.testutils.InputFormatTestUtil.writeDataBlockToLogFile; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -124,7 +125,7 @@ private void testReaderInternal(boolean partitioned, HoodieLogBlock.HoodieLogBlo HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), baseInstant, commitMetadata); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -163,7 +164,7 @@ private void testReaderInternal(boolean partitioned, HoodieLogBlock.HoodieLogBlo long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, commitMetadata); fileSlice.addLogFile(writer.getLogFile()); HoodieMergeOnReadSnapshotReader snapshotReader = new HoodieMergeOnReadSnapshotReader( diff --git a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java index b992987c6909..7999e72fffa8 100644 --- a/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java +++ b/hudi-hadoop-mr/src/test/java/org/apache/hudi/hadoop/realtime/TestHoodieRealtimeRecordReader.java @@ -100,6 +100,8 @@ import java.util.stream.Stream; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.hadoop.realtime.HoodieRealtimeRecordReader.REALTIME_SKIP_MERGE_PROP; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -203,7 +205,7 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), baseInstant, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), baseInstant, commitMetadata); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -240,7 +242,7 @@ private void testReaderInternal(ExternalSpillableMap.DiskMapType diskMapType, long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) fileSlice.addLogFile(writer.getLogFile()); @@ -315,7 +317,7 @@ public void testUnMergedReader() throws Exception { HoodieTableType.MERGE_ON_READ); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, commitMetadata); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -328,7 +330,7 @@ public void testUnMergedReader() throws Exception { long size = writer.getCurrentSize(); writer.close(); assertTrue(size > 0, "block - size should be > 0"); - FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), newCommitTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -399,7 +401,7 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); @@ -412,7 +414,7 @@ public void testReaderWithNestedAndComplexSchema(ExternalSpillableMap.DiskMapTyp assertTrue(size > 0, "block - size should be > 0"); commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), newCommitTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), newCommitTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -540,7 +542,7 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa instantTime, HoodieTableType.MERGE_ON_READ); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); List firstSchemaFields = schema.getFields(); @@ -570,7 +572,7 @@ public void testSchemaEvolutionAndRollbackBlockInLastLogFile(ExternalSpillableMa CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.DELTA_COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, commitMetadata); // create a split with baseFile (parquet file written earlier) and new log file(s) HoodieRealtimeFileSplit split = new HoodieRealtimeFileSplit( @@ -631,7 +633,7 @@ public void testSchemaEvolution() throws Exception { instantTime, HoodieTableType.MERGE_ON_READ); HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(basePath.toString(), instantTime, Option.of(commitMetadata)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, Option.of(commitMetadata)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir.getPath()); List firstSchemaFields = schema.getFields(); @@ -645,7 +647,7 @@ public void testSchemaEvolution() throws Exception { instantTime, HoodieTableType.MERGE_ON_READ, "2017", "05", "01"); HoodieCommitMetadata commitMetadata1 = CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, evolvedSchema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createCommit(basePath.toString(), newCommitTime, Option.of(commitMetadata1)); + FileCreateUtils.createCommit(COMMIT_METADATA_SER_DE, basePath.toString(), newCommitTime, Option.of(commitMetadata1)); // Add the paths FileInputFormat.setInputPaths(baseJobConf, partitionDir1.getPath()); @@ -801,7 +803,7 @@ private void createReplaceCommitFile( .resolve(commitNumber + "_" + InProcessTimeGenerator.createNewInstantTime() + ".replacecommit").toFile(); file.createNewFile(); FileOutputStream fileOutputStream = new FileOutputStream(file); - fileOutputStream.write(serializeCommitMetadata(replaceMetadata).get()); + fileOutputStream.write(serializeCommitMetadata(COMMIT_METADATA_SER_DE, replaceMetadata).get()); fileOutputStream.flush(); fileOutputStream.close(); } @@ -844,7 +846,7 @@ private void createDeltaCommitFile( .resolve(commitNumber + "_" + InProcessTimeGenerator.createNewInstantTime() + ".deltacommit").toFile(); file.createNewFile(); FileOutputStream fileOutputStream = new FileOutputStream(file); - fileOutputStream.write(serializeCommitMetadata(commitMetadata).get()); + fileOutputStream.write(serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); fileOutputStream.flush(); fileOutputStream.close(); } @@ -879,7 +881,7 @@ public void testLogOnlyReader() throws Exception { CommitUtils.buildMetadata(Collections.emptyList(), Collections.emptyMap(), Option.empty(), WriteOperationType.UPSERT, schema.toString(), HoodieTimeline.COMMIT_ACTION); - FileCreateUtils.createDeltaCommit(basePath.toString(), instantTime, commitMetadata); + FileCreateUtils.createDeltaCommit(COMMIT_METADATA_SER_DE, basePath.toString(), instantTime, commitMetadata); // create a split with new log file(s) fileSlice.addLogFile(new HoodieLogFile(writer.getLogFile().getPath(), size)); RealtimeFileStatus realtimeFileStatus = new RealtimeFileStatus( @@ -976,7 +978,7 @@ public void testAvroToArrayWritable() throws IOException { private File createCompactionFile(java.nio.file.Path basePath, String commitTime) throws IOException { File file = basePath.resolve(".hoodie") - .resolve(HoodieTimeline.makeRequestedCompactionFileName(commitTime)).toFile(); + .resolve(INSTANT_FILE_NAME_GENERATOR.makeRequestedCompactionFileName(commitTime)).toFile(); assertTrue(file.createNewFile()); FileOutputStream os = new FileOutputStream(file); try { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java index 38549b10cd39..ebacd1a7c9f2 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/HoodieTestSuiteJob.java @@ -23,9 +23,9 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.exception.HoodieException; import org.apache.hudi.hadoop.fs.HadoopFSUtils; @@ -151,10 +151,12 @@ public HoodieTestSuiteJob(HoodieTestSuiteConfig cfg, JavaSparkContext jsc, boole int getSchemaVersionFromCommit(int nthCommit) throws Exception { int version = 0; try { - HoodieTimeline timeline = new HoodieActiveTimeline(metaClient).getCommitsTimeline(); + HoodieTimeline timeline = new ActiveTimelineV2(metaClient).getCommitsTimeline(); // Pickup the schema version from nth commit from last (most recent insert/upsert will be rolled back). HoodieInstant prevInstant = timeline.nthFromLastInstant(nthCommit).get(); - HoodieCommitMetadata commit = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(prevInstant).get(), + HoodieCommitMetadata commit = metaClient.getCommitMetadataSerDe().deserialize( + prevInstant, + timeline.getInstantDetails(prevInstant).get(), HoodieCommitMetadata.class); Map extraMetadata = commit.getExtraMetadata(); String avroSchemaStr = extraMetadata.get(HoodieCommitMetadata.SCHEMA_KEY); diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java index 110eb091dcf5..70ac8c7f6c10 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/BaseValidateDatasetNode.java @@ -24,6 +24,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.exception.HoodieIOException; @@ -199,8 +200,9 @@ private void awaitUntilDeltaStreamerCaughtUp(ExecutionContext context, String hu private Option getLatestCheckpoint(HoodieTimeline timeline) { return (Option) timeline.getReverseOrderedInstants().map(instant -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() + .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_KEY))) { return Option.of(commitMetadata.getMetadata(CHECKPOINT_KEY)); } else { diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java index 5b96fbe5f8f1..2b6b60c9cd72 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/CompactNode.java @@ -55,8 +55,8 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E .getWriteTimeline().filterPendingCompactionTimeline().lastInstant(); if (lastInstant.isPresent()) { log.info("Compacting instant {}", lastInstant.get()); - this.result = executionContext.getHoodieTestSuiteWriter().compact(Option.of(lastInstant.get().getTimestamp())); - executionContext.getHoodieTestSuiteWriter().commitCompaction(result, executionContext.getJsc().emptyRDD(), Option.of(lastInstant.get().getTimestamp())); + this.result = executionContext.getHoodieTestSuiteWriter().compact(Option.of(lastInstant.get().requestedTime())); + executionContext.getHoodieTestSuiteWriter().commitCompaction(result, executionContext.getJsc().emptyRDD(), Option.of(lastInstant.get().requestedTime())); } } } diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java index 2b081e758660..377f3f9b30c5 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/RollbackNode.java @@ -71,7 +71,7 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E .equalsIgnoreCase(DFSTestSuitePathSelector.class.getName()), "Test Suite only supports DFSTestSuitePathSelector"); executionContext.getHoodieTestSuiteWriter().getWriteClient(this) - .rollback(lastInstant.get().getTimestamp()); + .rollback(lastInstant.get().requestedTime()); metaClient.getStorage().deleteDirectory(new StoragePath( executionContext.getWriterContext().getCfg().inputBasePath, executionContext.getWriterContext().getHoodieTestSuiteWriter().getLastCheckpoint() diff --git a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java index f6271cdfdf1d..d32e911172c5 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java @@ -48,8 +48,8 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E .build(); Option lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant(); if (lastInstant.isPresent()) { - HoodieCommitMetadata metadata = org.apache.hudi.common.model.HoodieCommitMetadata.fromBytes(metaClient - .getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize( + lastInstant.get(), metaClient.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); Option scheduledInstant = executionContext.getHoodieTestSuiteWriter().scheduleCompaction(Option.of(metadata .getExtraMetadata())); if (scheduledInstant.isPresent()) { 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 cbede15648cc..dc6698ede965 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 @@ -21,7 +21,6 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; @@ -38,6 +37,9 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * Node to validate data set sanity like total file versions retained, has cleaning happened, has archival happened, etc. */ @@ -69,7 +71,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)).countInstants(); + compareTimestamps(instant.requestedTime(), 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/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java index b8e02f06a416..3dff6ad9deeb 100644 --- a/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java +++ b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/reader/DFSHoodieDatasetInputReader.java @@ -290,7 +290,7 @@ private Iterator readColumnarOrLogFiles(FileSlice fileSlice) thro .collect(Collectors.toList())) .withReaderSchema(new Schema.Parser().parse(schemaStr)) .withLatestInstantTime(metaClient.getActiveTimeline().getCommitsTimeline() - .filterCompletedInstants().lastInstant().get().getTimestamp()) + .filterCompletedInstants().lastInstant().get().requestedTime()) .withMaxMemorySizeInBytes( HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES) .withReverseReader(false) diff --git a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java index e62c7defecf2..be7d866df92d 100644 --- a/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java +++ b/hudi-kafka-connect/src/main/java/org/apache/hudi/connect/utils/KafkaConnectUtils.java @@ -208,7 +208,7 @@ public static Option getCommitMetadataForLatestInstant(Hoo if (latestInstant.isPresent()) { try { byte[] data = timeline.getInstantDetails(latestInstant.get()).get(); - return Option.of(HoodieCommitMetadata.fromBytes(data, HoodieCommitMetadata.class)); + return Option.of(metaClient.getCommitMetadataSerDe().deserialize(latestInstant.get(), data, HoodieCommitMetadata.class)); } catch (Exception e) { throw new HoodieException("Failed to read schema from commit metadata", e); } diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java index ddf8d7ecc9d7..b1b720a97b15 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/common/table/timeline/HoodieMetaserverBasedTimeline.java @@ -21,6 +21,9 @@ import org.apache.hudi.common.config.HoodieMetaserverConfig; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.versioning.v2.ActiveTimelineV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantGeneratorV2; +import org.apache.hudi.common.table.timeline.versioning.v2.InstantFileNameGeneratorV2; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; @@ -31,12 +34,14 @@ /** * Active timeline for hoodie table whose metadata is stored in the hoodie meta server instead of file system. + * Note. MetadataServer only works with 1.x table version and will be disabled when in prior table version. */ -public class HoodieMetaserverBasedTimeline extends HoodieActiveTimeline { +public class HoodieMetaserverBasedTimeline extends ActiveTimelineV2 { private final String databaseName; private final String tableName; private final HoodieMetaserverClient metaserverClient; - + private final InstantGeneratorV2 instantGenerator = new InstantGeneratorV2(); + private final InstantFileNameGeneratorV2 instantFileNameGenerator = new InstantFileNameGeneratorV2(); public HoodieMetaserverBasedTimeline(HoodieTableMetaClient metaClient, HoodieMetaserverConfig config) { this.metaClient = metaClient; this.metaserverClient = HoodieMetaserverClientProxy.getProxy(config); @@ -52,20 +57,20 @@ protected void deleteInstantFile(HoodieInstant instant) { @Override protected void transitionStateToComplete(boolean shouldLock, HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { - ValidationUtils.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp())); + ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime())); metaserverClient.transitionInstantState(databaseName, tableName, fromInstant, toInstant, data); } @Override public void transitionPendingState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data, boolean allowRedundantTransitions) { - ValidationUtils.checkArgument(fromInstant.getTimestamp().equals(toInstant.getTimestamp())); + ValidationUtils.checkArgument(fromInstant.requestedTime().equals(toInstant.requestedTime())); metaserverClient.transitionInstantState(databaseName, tableName, fromInstant, toInstant, data); } @Override public void createFileInMetaPath(String filename, Option content, boolean allowOverwrite) { StoragePathInfo pathInfo = new StoragePathInfo(new StoragePath(filename), 0, false, (short) 0, 0, 0); - HoodieInstant instant = new HoodieInstant(pathInfo); + HoodieInstant instant = instantGenerator.createNewInstant(pathInfo); ValidationUtils.checkArgument(instant.getState().equals(HoodieInstant.State.REQUESTED)); metaserverClient.createNewInstant(databaseName, tableName, instant, Option.empty()); } @@ -78,7 +83,7 @@ protected void revertCompleteToInflight(HoodieInstant completed, HoodieInstant i @Override protected Option readDataFromPath(StoragePath detailPath) { StoragePathInfo pathInfo = new StoragePathInfo(detailPath, 0, false, (short) 0, 0, 0); - HoodieInstant instant = new HoodieInstant(pathInfo); + HoodieInstant instant = instantGenerator.createNewInstant(pathInfo); return metaserverClient.getInstantMetadata(databaseName, tableName, instant); } @@ -95,9 +100,9 @@ public HoodieMetaserverBasedTimeline reload() { protected String getInstantFileName(HoodieInstant instant) { if (instant.isCompleted()) { // Set a fake completion time. - return instant.getFileName("0").replace("_0", ""); + return instantFileNameGenerator.getFileName("0", instant).replace("_0", ""); } - return instant.getFileName(); + return instantFileNameGenerator.getFileName(instant); } } diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java index 56a6e6b2df66..2c8be6c30aab 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/client/HoodieMetaserverClientImp.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.config.HoodieMetaserverConfig; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.versioning.DefaultInstantGenerator; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.RetryHelper; @@ -56,6 +57,7 @@ public class HoodieMetaserverClientImp implements HoodieMetaserverClient { private boolean isLocal; private ThriftHoodieMetaserver.Iface client; private TTransport transport; + private DefaultInstantGenerator instantGenerator = new DefaultInstantGenerator(); public HoodieMetaserverClientImp(HoodieMetaserverConfig config) { this.config = config; @@ -110,8 +112,8 @@ public void createTable(Table table) { @Override public List listInstants(String db, String tb, int commitNum) { return exceptionWrapper(() -> this.client.listInstants(db, tb, commitNum).stream() - .map(EntityConversions::fromTHoodieInstant) - .sorted(Comparator.comparing(HoodieInstant::getTimestamp).reversed()) + .map(instant -> EntityConversions.fromTHoodieInstant(instant, instantGenerator)) + .sorted(Comparator.comparing(HoodieInstant::requestedTime).reversed()) .collect(Collectors.toList())).get(); } diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/util/EntityConversions.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/util/EntityConversions.java index 07214dd14030..5eb7c605ab4d 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/util/EntityConversions.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-client/src/main/java/org/apache/hudi/metaserver/util/EntityConversions.java @@ -20,6 +20,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantGenerator; import org.apache.hudi.metaserver.thrift.TAction; import org.apache.hudi.metaserver.thrift.THoodieInstant; import org.apache.hudi.metaserver.thrift.TState; @@ -32,11 +33,11 @@ public class EntityConversions { public static THoodieInstant toTHoodieInstant(HoodieInstant instant) { - return new THoodieInstant(instant.getTimestamp(), toTAction(instant.getAction()), toTState(instant.getState())); + return new THoodieInstant(instant.requestedTime(), toTAction(instant.getAction()), toTState(instant.getState())); } - public static HoodieInstant fromTHoodieInstant(THoodieInstant instant) { - return new HoodieInstant(fromTState(instant.getState()), fromTAction(instant.getAction()), instant.getTimestamp()); + public static HoodieInstant fromTHoodieInstant(THoodieInstant instant, InstantGenerator instantGenerator) { + return instantGenerator.createNewInstant(fromTState(instant.getState()), fromTAction(instant.getAction()), instant.getTimestamp()); } public static TAction toTAction(String action) { diff --git a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/java/org/apache/hudi/metaserver/store/RelationalDBBasedStorage.java b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/java/org/apache/hudi/metaserver/store/RelationalDBBasedStorage.java index b0b8bfc1403e..8f3fde26f730 100644 --- a/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/java/org/apache/hudi/metaserver/store/RelationalDBBasedStorage.java +++ b/hudi-platform-service/hudi-metaserver/hudi-metaserver-server/src/main/java/org/apache/hudi/metaserver/store/RelationalDBBasedStorage.java @@ -18,9 +18,7 @@ package org.apache.hudi.metaserver.store; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.metaserver.store.bean.InstantBean; import org.apache.hudi.metaserver.store.bean.TableBean; @@ -38,6 +36,8 @@ import java.util.Map; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.CollectionUtils.isNullOrEmpty; import static org.apache.hudi.common.util.ValidationUtils.checkState; @@ -128,8 +128,8 @@ public String createNewTimestamp(long tableId) throws MetaserverStorageException oldTimestamp = getLatestTimestamp(tableId); do { newTimestamp = HoodieInstantTimeGenerator.formatDate(new Date(System.currentTimeMillis())); - } while (oldTimestamp != null && HoodieTimeline.compareTimestamps(newTimestamp, - HoodieActiveTimeline.LESSER_THAN_OR_EQUALS, oldTimestamp)); + } while (oldTimestamp != null && compareTimestamps(newTimestamp, + LESSER_THAN_OR_EQUALS, oldTimestamp)); Map params = new HashMap<>(); params.put("tableId", tableId); params.put("oldTimestamp", oldTimestamp); diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteCommitActionExecutor.java index 01e7315cb529..e68e818bedee 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteCommitActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/DatasetBulkInsertOverwriteCommitActionExecutor.java @@ -50,7 +50,8 @@ public DatasetBulkInsertOverwriteCommitActionExecutor(HoodieWriteConfig config, @Override protected Option> doExecute(Dataset records, boolean arePartitionRecordsSorted) { - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, getCommitActionType(), instantTime), Option.empty()); + table.getActiveTimeline().transitionRequestedToInflight(table.getMetaClient().createNewInstant(HoodieInstant.State.REQUESTED, + getCommitActionType(), instantTime), Option.empty()); return Option.of(HoodieDatasetBulkInsertHelper .bulkInsert(records, instantTime, table, writeConfig, arePartitionRecordsSorted, false)); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java index 2a5113538e4d..faaadaeb497b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/commit/HoodieStreamerDatasetBulkInsertCommitActionExecutor.java @@ -48,7 +48,8 @@ protected void preExecute() { @Override protected Option> doExecute(Dataset records, boolean arePartitionRecordsSorted) { - table.getActiveTimeline().transitionRequestedToInflight(new HoodieInstant(HoodieInstant.State.REQUESTED, getCommitActionType(), instantTime), Option.empty()); + table.getActiveTimeline().transitionRequestedToInflight(table.getMetaClient().createNewInstant(HoodieInstant.State.REQUESTED, + getCommitActionType(), instantTime), Option.empty()); return Option.of(HoodieDatasetBulkInsertHelper .bulkInsert(records, instantTime, table, writeConfig, arePartitionRecordsSorted, false)); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java index d37dd7f7c2ae..1c9e2bb75795 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java +++ b/hudi-spark-datasource/hudi-spark-common/src/main/java/org/apache/hudi/internal/DataSourceInternalWriterHelper.java @@ -24,7 +24,6 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; @@ -101,9 +100,8 @@ public void abort() { public void createInflightCommit() { metaClient.getActiveTimeline().transitionRequestedToInflight( - new HoodieInstant(State.REQUESTED, - CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), - instantTime), Option.empty()); + metaClient.createNewInstant(State.REQUESTED, + CommitUtils.getCommitActionType(operationType, metaClient.getTableType()), instantTime), Option.empty()); } public HoodieTable getHoodieTable() { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala index c4fdd678d11b..d359fcc55506 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieBaseRelation.scala @@ -18,7 +18,7 @@ package org.apache.hudi import org.apache.hudi.AvroConversionUtils.getAvroSchemaWithDefaults -import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, createHFileReader, isSchemaEvolutionEnabledOnRead, metaFieldNames, projectSchema, sparkAdapter, BaseFileReader} +import org.apache.hudi.HoodieBaseRelation.{BaseFileReader, convertToAvroSchema, createHFileReader, isSchemaEvolutionEnabledOnRead, metaFieldNames, projectSchema, sparkAdapter} import org.apache.hudi.HoodieConversionUtils.toScalaOption import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.client.utils.SparkInternalSchemaConverter @@ -30,7 +30,7 @@ import org.apache.hudi.common.model.{FileSlice, HoodieFileFormat, HoodieRecord} import org.apache.hudi.common.model.HoodieFileFormat.HFILE import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.HoodieTimeline +import org.apache.hudi.common.table.timeline.{HoodieTimeline, TimelineLayout} import org.apache.hudi.common.table.timeline.TimelineUtils.validateTimestampAsOf import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.util.{ConfigUtils, StringUtils} @@ -48,7 +48,6 @@ import org.apache.hudi.io.storage.HoodieSparkIOFactory import org.apache.hudi.metadata.HoodieTableMetadata import org.apache.hudi.storage.{StoragePath, StoragePathInfo} import org.apache.hudi.storage.hadoop.HoodieHadoopStorage - import org.apache.avro.Schema import org.apache.avro.generic.GenericRecord import org.apache.hadoop.conf.Configuration @@ -57,7 +56,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SparkSession, SQLContext} +import org.apache.spark.sql.{Row, SQLContext, SparkSession} import org.apache.spark.sql.HoodieCatalystExpressionUtils.{convertToCatalystExpression, generateUnsafeProjection} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.Resolver @@ -292,7 +291,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants private def queryTimestamp: Option[String] = - specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.getTimestamp)) + specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.requestedTime)) /** * Returns true in case table supports Schema on Read (Schema Evolution) @@ -680,7 +679,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext, val internalSchema = internalSchemaOpt.getOrElse(InternalSchema.getEmptyInternalSchema) val querySchemaString = SerDeHelper.toJson(internalSchema) if (!isNullOrEmpty(querySchemaString)) { - val validCommits = timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",") + val instantFileNameGenerator = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion).getInstantFileNameGenerator + val validCommits = timeline.getInstants.iterator.asScala.map(instant => instantFileNameGenerator.getFileName(instant)).mkString(",") conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema)) conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath.toString) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala index c856f106ee4b..ceb5a3826824 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieHadoopFsRelationFactory.scala @@ -138,7 +138,8 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, } protected lazy val validCommits: String = if (internalSchemaOpt.nonEmpty) { - timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",") + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator + timeline.getInstants.iterator.asScala.map(instant => instantFileNameGenerator.getFileName(instant)).mkString(",") } else { "" } @@ -200,7 +201,7 @@ abstract class HoodieBaseHadoopFsRelationFactory(val sqlContext: SQLContext, protected lazy val shouldUseRecordPosition: Boolean = checkIfAConfigurationEnabled(HoodieReaderConfig.MERGE_USE_RECORD_POSITIONS) protected def queryTimestamp: Option[String] = - specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.getTimestamp)) + specifiedQueryTimestamp.orElse(toScalaOption(timeline.lastInstant()).map(_.requestedTime)) protected def hasSchemaOnRead: Boolean = internalSchemaOpt.isDefined diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala index 1aaef8f55af3..0e403543f740 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieStreamingSink.scala @@ -33,7 +33,6 @@ import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.config.HoodieWriteConfig.WRITE_CONCURRENCY_MODE import org.apache.hudi.exception.{HoodieCorruptedDataException, HoodieException, TableNotFoundException} import org.apache.hudi.hadoop.fs.HadoopFSUtils - import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.execution.streaming.{Sink, StreamExecution} import org.apache.spark.sql.streaming.OutputMode @@ -42,7 +41,6 @@ import org.slf4j.LoggerFactory import java.lang import java.util.function.{BiConsumer, Function} - import scala.collection.JavaConverters._ import scala.util.{Failure, Success, Try} @@ -158,13 +156,10 @@ class HoodieStreamingSink(sqlContext: SQLContext, .build()) } if (compactionInstantOps.isPresent) { - asyncCompactorService.enqueuePendingAsyncServiceInstant( - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionInstantOps.get())) + asyncCompactorService.enqueuePendingAsyncServiceInstant(compactionInstantOps.get()) } if (clusteringInstant.isPresent) { - asyncClusteringService.enqueuePendingAsyncServiceInstant(new HoodieInstant( - State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusteringInstant.get() - )) + asyncClusteringService.enqueuePendingAsyncServiceInstant(clusteringInstant.get()) } Success((true, commitOps, compactionInstantOps)) case Failure(e) => @@ -272,7 +267,7 @@ class HoodieStreamingSink(sqlContext: SQLContext, .setBasePath(client.getConfig.getBasePath).build() val pendingInstants: java.util.List[HoodieInstant] = CompactionUtils.getPendingCompactionInstantTimes(metaClient) - pendingInstants.asScala.foreach((h: HoodieInstant) => asyncCompactorService.enqueuePendingAsyncServiceInstant(h)) + pendingInstants.asScala.foreach((h: HoodieInstant) => asyncCompactorService.enqueuePendingAsyncServiceInstant(h.requestedTime())) } } @@ -300,7 +295,7 @@ class HoodieStreamingSink(sqlContext: SQLContext, .setConf(HadoopFSUtils.getStorageConfWithCopy(sqlContext.sparkContext.hadoopConfiguration)) .setBasePath(client.getConfig.getBasePath).build() val pendingInstants: java.util.List[HoodieInstant] = ClusteringUtils.getPendingClusteringInstantTimes(metaClient) - pendingInstants.asScala.foreach((h: HoodieInstant) => asyncClusteringService.enqueuePendingAsyncServiceInstant(h)) + pendingInstants.asScala.foreach((h: HoodieInstant) => asyncClusteringService.enqueuePendingAsyncServiceInstant(h.requestedTime())) } } 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 38cb2cecf25d..54f56a26f82b 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 @@ -101,7 +101,7 @@ class IncrementalRelation(val sqlContext: SQLContext, val iSchema : InternalSchema = if (!isSchemaEvolutionEnabledOnRead(optParams, sqlContext.sparkSession)) { InternalSchema.getEmptyInternalSchema } else if (useEndInstantSchema && !commitsToReturn.isEmpty) { - InternalSchemaCache.searchSchemaAndCache(commitsToReturn.last.getTimestamp.toLong, metaClient) + InternalSchemaCache.searchSchemaAndCache(commitsToReturn.last.requestedTime.toLong, metaClient) } else { schemaResolver.getTableInternalSchemaFromCommitMetadata.orElse(null) } @@ -153,10 +153,10 @@ class IncrementalRelation(val sqlContext: SQLContext, }.toMap for (commit <- commitsToReturn) { - val metadata: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(commit) - .get, classOf[HoodieCommitMetadata]) + val metadata: HoodieCommitMetadata = metaClient.getCommitMetadataSerDe.deserialize(commit, + commitTimeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) - if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.getTimestamp) { + if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.requestedTime) { metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).asScala.filterNot { case (k, v) => replacedFile.contains(k) && v.startsWith(replacedFile(k)) } @@ -186,8 +186,9 @@ class IncrementalRelation(val sqlContext: SQLContext, } } // pass internalSchema to hadoopConf, so it can be used in executors. + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator; val validCommits = metaClient - .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstantsAsStream.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",") + .getCommitsAndCompactionTimeline.filterCompletedInstants.getInstantsAsStream.toArray().map(a => instantFileNameGenerator.getFileName(a.asInstanceOf[HoodieInstant])).mkString(",") sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema)) sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath.toString) sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits) @@ -256,7 +257,7 @@ class IncrementalRelation(val sqlContext: SQLContext, if (regularFileIdToFullPath.nonEmpty) { try { - val commitTimesToReturn = commitsToReturn.map(_.getTimestamp) + val commitTimesToReturn = commitsToReturn.map(_.requestedTime) df = df.union(sqlContext.read.options(sOpts) .schema(usedSchema).format(formatClassName) // Setting time to the END_INSTANT_TIME, to avoid pathFilter filter out files incorrectly. @@ -282,7 +283,7 @@ class IncrementalRelation(val sqlContext: SQLContext, } private def fullTableScanDataFrame(commitsToFilter: List[HoodieInstant]): DataFrame = { - val commitTimesToFilter = commitsToFilter.map(_.getTimestamp) + val commitTimesToFilter = commitsToFilter.map(_.requestedTime) val hudiDF = sqlContext.read .format("hudi_v1") .schema(usedSchema) 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 6893b28a3ec2..ecb9b79bcb8a 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 @@ -93,7 +93,7 @@ case class MergeOnReadIncrementalRelation(override val sqlContext: SQLContext, tableState = tableState, mergeType = mergeType, fileSplits = fileSplits, - includedInstantTimeSet = Option(includedCommits.map(_.getTimestamp).toSet)) + includedInstantTimeSet = Option(includedCommits.map(_.requestedTime).toSet)) } override protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): List[HoodieMergeOnReadFileSplit] = { @@ -103,7 +103,7 @@ case class MergeOnReadIncrementalRelation(override val sqlContext: SQLContext, val fileSlices = if (fullTableScan) { listLatestFileSlices(Seq(), partitionFilters, dataFilters) } else { - val latestCommit = includedCommits.last.getTimestamp + val latestCommit = includedCommits.last.requestedTime val fsView = new HoodieTableFileSystemView( metaClient, timeline, affectedFilesInCommits) @@ -126,7 +126,7 @@ case class MergeOnReadIncrementalRelation(override val sqlContext: SQLContext, val fileSlices = if (fullTableScan) { listLatestFileSlices(Seq(), partitionFilters, dataFilters) } else { - val latestCommit = includedCommits.last.getTimestamp + val latestCommit = includedCommits.last.requestedTime val fsView = new HoodieTableFileSystemView(metaClient, timeline, affectedFilesInCommits) val modifiedPartitions = getWritePartitionPaths(commitsMetadata) @@ -220,7 +220,7 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation { protected lazy val incrementalSpanRecordFilters: Seq[Filter] = { val isNotNullFilter = IsNotNull(HoodieRecord.COMMIT_TIME_METADATA_FIELD) - val timeStamps = includedCommits.map(_.getTimestamp).toArray[Any] + val timeStamps = includedCommits.map(_.requestedTime).toArray[Any] val inFilter = In(HoodieRecord.COMMIT_TIME_METADATA_FIELD, timeStamps) Seq(isNotNullFilter, inFilter) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala index 708e0d47a0ab..51e0f8465513 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/RecordLevelIndexSupport.scala @@ -25,7 +25,8 @@ import org.apache.hudi.common.model.FileSlice import org.apache.hudi.common.model.HoodieRecord.HoodieMetadataField import org.apache.hudi.common.model.HoodieTableQueryType.SNAPSHOT import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.HoodieTimeline.{GREATER_THAN_OR_EQUALS, compareTimestamps} +import org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps +import org.apache.hudi.common.table.timeline.InstantComparison import org.apache.hudi.metadata.HoodieTableMetadataUtil import org.apache.hudi.storage.StoragePath import org.apache.spark.sql.SparkSession @@ -112,7 +113,7 @@ class RecordLevelIndexSupport(spark: SparkSession, // Check if the as.of.instant is greater than or equal to the last completed instant. // We can still use RLI for data skipping for the latest snapshot. compareTimestamps(HoodieSqlCommonUtils.formatQueryInstant(instant), - GREATER_THAN_OR_EQUALS, metaClient.getCommitsTimeline.filterCompletedInstants.lastInstant.get.getTimestamp) + InstantComparison.GREATER_THAN_OR_EQUALS, metaClient.getCommitsTimeline.filterCompletedInstants.lastInstant.get.requestedTime) } } } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/TimelineRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/TimelineRelation.scala index 9029993f354c..d0e8b67a459c 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/TimelineRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/TimelineRelation.scala @@ -84,6 +84,7 @@ class TimelineRelation(val sqlContext: SQLContext, var totalRecordsWritten: Long = -1 var totalUpdatedRecordsWritten: Long = -1 var totalWriteErrors: Long = -1 + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator; val commitMetadataOpt = CommitUtils.buildMetadataFromInstant(timeline, instant) if (commitMetadataOpt.isPresent) { @@ -96,11 +97,11 @@ class TimelineRelation(val sqlContext: SQLContext, totalWriteErrors = commitMetadata.fetchTotalWriteErrors } - val r = Row(instant.getTimestamp, + val r = Row(instant.requestedTime, instant.getAction, instant.getState.toString, instant.getCompletionTime, - instant.getFileName, + instantFileNameGenerator.getFileName(instant), totalBytesWritten, totalFilesUpdated, totalPartitionsWritten, diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala index cc694be9be7b..f4ec2b916eb6 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/cdc/CDCRelation.scala @@ -212,7 +212,7 @@ object CDCRelation { def getTimestampOfLatestInstant(metaClient: HoodieTableMetaClient): String = { val latestInstant = metaClient.getActiveTimeline.lastInstant() if (latestInstant.isPresent) { - latestInstant.get().getTimestamp + latestInstant.get().requestedTime } else { throw new HoodieException("No valid instant in Active Timeline.") } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala index 59daefa54266..29a9f97d1138 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/HoodieSqlCommonUtils.scala @@ -25,9 +25,9 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.HoodieRecord +import org.apache.hudi.common.table.timeline.TimelineUtils.parseDateFromInstantTime import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstantTimeGenerator, HoodieTimeline} -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline.parseDateFromInstantTime +import org.apache.hudi.common.table.timeline.{HoodieInstantTimeGenerator, HoodieTimeline, TimelineUtils} import org.apache.hudi.common.util.PartitionPathEncodeUtils import org.apache.hudi.exception.HoodieException import org.apache.hudi.storage.{HoodieStorage, StoragePath, StoragePathInfo} @@ -258,7 +258,7 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport { validateInstant(queryInstant) queryInstant } else if (instantLength == 10) { // for yyyy-MM-dd - HoodieActiveTimeline.formatDate(defaultDateFormat.get().parse(queryInstant)) + TimelineUtils.formatDate(defaultDateFormat.get().parse(queryInstant)) } else { throw new IllegalArgumentException(s"Unsupported query instant time format: $queryInstant," + s"Supported time format are: 'yyyy-MM-dd: HH:mm:ss.SSS' or 'yyyy-MM-dd' or 'yyyyMMddHHmmssSSS'") diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala index 3c8c63bed372..6dc12d865a0e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterHoodieTableAddColumnsCommand.scala @@ -21,7 +21,6 @@ import org.apache.avro.Schema import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, HoodieWriterUtils, SparkAdapterSupport} import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieTableType, WriteOperationType} -import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata import org.apache.hudi.common.util.CommitUtils @@ -111,10 +110,10 @@ object AlterHoodieTableAddColumnsCommand extends SparkAdapterSupport with Loggin val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext) val timeLine = hoodieTable.getActiveTimeline - val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime) + val requested = hoodieTable.getInstantGenerator.createNewInstant(State.REQUESTED, commitActionType, instantTime) val metadata = new HoodieCommitMetadata metadata.setOperationType(WriteOperationType.ALTER_SCHEMA) - timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(metadata)) + timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(hoodieTable.getMetaClient.getTimelineLayout.getCommitMetadataSerDe, metadata)) client.commit(instantTime, jsc.emptyRDD) } 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 7fb1c7cc1ed5..43a08aeb3108 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 @@ -64,14 +64,14 @@ public static List listCommitsSince(FileSystem fs, String basePath, String instantTimestamp) { HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath); return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .map(HoodieInstant::requestedTime).collect(Collectors.toList()); } public static List listCommitsSince(HoodieStorage storage, String basePath, String instantTimestamp) { HoodieTimeline timeline = allCompletedCommitsCompactions(storage, basePath); return timeline.findInstantsAfter(instantTimestamp, Integer.MAX_VALUE).getInstantsAsStream() - .map(HoodieInstant::getTimestamp).collect(Collectors.toList()); + .map(HoodieInstant::requestedTime).collect(Collectors.toList()); } // this is used in the integration test script: docker/demo/sparksql-incremental.commands @@ -90,12 +90,12 @@ public static List listCompletionTimeSince(FileSystem fs, String basePat @PublicAPIMethod(maturity = ApiMaturityLevel.STABLE) public static String latestCommit(FileSystem fs, String basePath) { HoodieTimeline timeline = allCompletedCommitsCompactions(fs, basePath); - return timeline.lastInstant().get().getTimestamp(); + return timeline.lastInstant().get().requestedTime(); } public static String latestCommit(HoodieStorage storage, String basePath) { HoodieTimeline timeline = allCompletedCommitsCompactions(storage, basePath); - return timeline.lastInstant().get().getTimestamp(); + return timeline.lastInstant().get().requestedTime(); } /** @@ -140,7 +140,7 @@ public static Option getClusteringPlan(FileSystem fs, Stri HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder() .setConf(HadoopFSUtils.getStorageConfWithCopy(fs.getConf())) .setBasePath(basePath).setLoadActiveTimelineOnLoad(true).build(); - Option hoodieInstant = metaClient.getActiveTimeline().filter(instant -> instant.getTimestamp().equals(instantTime) + Option hoodieInstant = metaClient.getActiveTimeline().filter(instant -> instant.requestedTime().equals(instantTime) && ClusteringUtils.isClusteringOrReplaceCommitAction(instant.getAction())) .firstInstant(); Option> clusteringPlan = diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java index 8ffb2991386c..da30c4c865bd 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java @@ -18,11 +18,16 @@ package org.apache.hudi.cli; +import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.TimelineArchivers; import org.apache.hudi.common.config.HoodieMetadataConfig; +import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; +import org.apache.hudi.common.model.HoodieKey; +import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieWriteConfig; @@ -60,7 +65,8 @@ public static int archive(JavaSparkContext jsc, HoodieSparkTable table = HoodieSparkTable.create(config, context); CommonClientUtils.validateTableVersion(table.getMetaClient().getTableConfig(), config); try { - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(config, table); + HoodieTimelineArchiver>, HoodieData, HoodieData> archiver = + TimelineArchivers.getInstance(table.getMetaClient().getTimelineLayoutVersion(), config, table); archiver.archiveIfRequired(context, true); } catch (IOException ioe) { LOG.error("Failed to archive with IOException: {}", ioe.getMessage()); 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 3092df255251..928a2c4fb0f5 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 @@ -18,13 +18,12 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils -import org.apache.hudi.common.table.timeline.HoodieTimeline - +import org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps +import org.apache.hudi.common.table.timeline.InstantComparison import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util.function.Supplier - import scala.collection.JavaConverters._ class CommitsCompareProcedure() extends BaseProcedure with ProcedureBuilder { @@ -54,15 +53,15 @@ 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.requestedTime else "0" val sourceLatestCommit = - if (sourceTimeline.getInstants.iterator().hasNext) sourceTimeline.lastInstant.get.getTimestamp else "0" + if (sourceTimeline.getInstants.iterator().hasNext) sourceTimeline.lastInstant.get.requestedTime 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 + if (sourceLatestCommit != null && compareTimestamps(targetLatestCommit, InstantComparison.GREATER_THAN, sourceLatestCommit)) { // source is behind the target + val commitsToCatchup = targetTimeline.findInstantsAfter(sourceLatestCommit, Integer.MAX_VALUE).getInstants.iterator().asScala.map(instant => instant.requestedTime).toList.asJava Seq(Row("Source " + source.getTableConfig.getTableName + " is behind by " + commitsToCatchup.size + " commits. Commits to catch up - " + commitsToCatchup)) } else { - val commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants.iterator().asScala.map(instant => instant.getTimestamp).toList.asJava + val commitsToCatchup = sourceTimeline.findInstantsAfter(targetLatestCommit, Integer.MAX_VALUE).getInstants.iterator().asScala.map(instant => instant.requestedTime).toList.asJava Seq(Row("Source " + source.getTableConfig.getTableName + " is ahead by " + commitsToCatchup.size + " commits. Commits to catch up - " + commitsToCatchup)) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointProcedure.scala index 0ae22f54af7f..e48d27db6c80 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/CreateSavepointProcedure.scala @@ -59,7 +59,7 @@ class CreateSavepointProcedure extends BaseProcedure with ProcedureBuilder with val completedTimeline: HoodieTimeline = metaClient.getCommitsTimeline.filterCompletedInstants if (StringUtils.isNullOrEmpty(commitTime)) { - commitTime = completedTimeline.lastInstant.get.getTimestamp + commitTime = completedTimeline.lastInstant.get.requestedTime } else if (!completedTimeline.containsInstant(commitTime)) { throw new HoodieException("Commit " + commitTime + " not found in Commits " + completedTimeline) } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala index d9a6dc4197d2..bc350ffe632f 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/DeleteSavepointProcedure.scala @@ -56,7 +56,7 @@ class DeleteSavepointProcedure extends BaseProcedure with ProcedureBuilder with val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete") if (StringUtils.isNullOrEmpty(instantTime)) { - instantTime = completedInstants.lastInstant.get.getTimestamp + instantTime = completedInstants.lastInstant.get.requestedTime } val instantTimes = instantTime.split(",") val client = HoodieCLIUtils.createHoodieWriteClient(sparkSession, basePath, Map.empty, @@ -64,7 +64,8 @@ class DeleteSavepointProcedure extends BaseProcedure with ProcedureBuilder with var result = true var currentInstant = "" for (it <- instantTimes) { - val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, it) + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator + val savePoint = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, it) currentInstant = it if (!completedInstants.containsInstant(savePoint)) { throw new HoodieException("Commit " + it + " not found in Commits " + completedInstants) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala index f44b478f3f86..dd6173f2f283 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ExportInstantsProcedure.scala @@ -181,8 +181,9 @@ class ExportInstantsProcedure extends BaseProcedure with ProcedureBuilder with L if (!instants.isEmpty) { val timeline = metaClient.getActiveTimeline val storage = HoodieStorageUtils.getStorage(metaClient.getBasePath, HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration())) + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator for (instant <- instants.asScala) { - val localPath = localFolder + StoragePath.SEPARATOR + instant.getFileName + val localPath = localFolder + StoragePath.SEPARATOR + instantFileNameGenerator.getFileName(instant) val data: Array[Byte] = instant.getAction match { case HoodieTimeline.CLEAN_ACTION => val metadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(instant).get) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala index 1eadad0187d5..90c145220dcb 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairAddpartitionmetaProcedure.scala @@ -55,7 +55,7 @@ class RepairAddpartitionmetaProcedure extends BaseProcedure with ProcedureBuilde val metaClient = createMetaClient(jsc, tablePath) - val latestCommit: String = metaClient.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.getTimestamp + val latestCommit: String = metaClient.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.requestedTime val partitionPaths: util.List[String] = FSUtils.getAllPartitionFoldersThreeLevelsDown(metaClient.getStorage, tablePath); val basePath: StoragePath = new StoragePath(tablePath) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala index b0bc57ab7c17..124324ac3f9c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairCorruptedCleanFilesProcedure.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.hudi.command.procedures -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant} +import org.apache.hudi.common.table.timeline.{HoodieInstant, TimelineUtils} import org.apache.hudi.common.util.CleanerUtils import org.apache.hudi.exception.HoodieIOException - import org.apache.avro.AvroRuntimeException import org.apache.spark.internal.Logging import org.apache.spark.sql.Row @@ -28,7 +27,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.io.IOException import java.util.function.Supplier - import scala.collection.JavaConverters.asScalaIteratorConverter class RepairCorruptedCleanFilesProcedure extends BaseProcedure with ProcedureBuilder with Logging { @@ -51,7 +49,7 @@ class RepairCorruptedCleanFilesProcedure extends BaseProcedure with ProcedureBui val tablePath = getBasePath(tableName) val metaClient = createMetaClient(jsc, tablePath) - + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator val cleanerTimeline = metaClient.getActiveTimeline.getCleanerTimeline logInfo("Inspecting pending clean metadata in timeline for corrupted files") var result = true @@ -61,11 +59,11 @@ class RepairCorruptedCleanFilesProcedure extends BaseProcedure with ProcedureBui } catch { case e: AvroRuntimeException => logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) - HoodieActiveTimeline.deleteInstantFile(metaClient.getStorage, metaClient.getMetaPath, instant) + TimelineUtils.deleteInstantFile(metaClient.getStorage, metaClient.getMetaPath, instant, instantFileNameGenerator) case ioe: IOException => if (ioe.getMessage.contains("Not an Avro data file")) { logWarning("Corruption found. Trying to remove corrupted clean instant file: " + instant) - HoodieActiveTimeline.deleteInstantFile(metaClient.getStorage, metaClient.getMetaPath, instant) + TimelineUtils.deleteInstantFile(metaClient.getStorage, metaClient.getMetaPath, instant, instantFileNameGenerator) } else { result = false throw new HoodieIOException(ioe.getMessage, ioe) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala index b47f434e6094..3cedd1a9cba3 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RepairMigratePartitionMetaProcedure.scala @@ -73,7 +73,7 @@ class RepairMigratePartitionMetaProcedure extends BaseProcedure with ProcedureBu metaClient.getStorage, partition) val baseFormatFile: Option[StoragePath] = HoodiePartitionMetadata.baseFormatMetaPathIfExists( metaClient.getStorage, partition) - val latestCommit: String = metaClient.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.getTimestamp + val latestCommit: String = metaClient.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.requestedTime var action = if (textFormatFile.isPresent) "MIGRATE" else "NONE" if (!dryRun) { if (!baseFormatFile.isPresent) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToSavepointProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToSavepointProcedure.scala index 80688838bd2b..e5d32209640b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToSavepointProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RollbackToSavepointProcedure.scala @@ -52,13 +52,13 @@ class RollbackToSavepointProcedure extends BaseProcedure with ProcedureBuilder w val basePath: String = getBasePath(tableName, tablePath) val metaClient = createMetaClient(jsc, basePath) - + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val completedInstants = metaClient.getActiveTimeline.getSavePointTimeline.filterCompletedInstants if (completedInstants.empty) throw new HoodieException("There are no completed savepoint to run delete") if (StringUtils.isNullOrEmpty(instantTime)) { - instantTime = completedInstants.lastInstant.get.getTimestamp + instantTime = completedInstants.lastInstant.get.requestedTime } - val savePoint = new HoodieInstant(false, HoodieTimeline.SAVEPOINT_ACTION, instantTime) + val savePoint = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, instantTime) if (!completedInstants.containsInstant(savePoint)) { throw new HoodieException("Commit " + instantTime + " not found in Commits " + completedInstants) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala index 9fc3d041daae..52f7206e928c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunClusteringProcedure.scala @@ -141,7 +141,7 @@ class RunClusteringProcedure extends BaseProcedure } val pendingClusteringInstants = ClusteringUtils.getAllPendingClusteringPlans(metaClient) - .iterator().asScala.map(_.getLeft.getTimestamp).toSeq.sortBy(f => f) + .iterator().asScala.map(_.getLeft.requestedTime).toSeq.sortBy(f => f) var (filteredPendingClusteringInstants, operation) = HoodieProcedureUtils.filterPendingInstantsAndGetOperation( pendingClusteringInstants, specificInstants.asInstanceOf[Option[String]], op.asInstanceOf[Option[String]], limit.asInstanceOf[Option[Int]]) @@ -171,9 +171,9 @@ class RunClusteringProcedure extends BaseProcedure } val clusteringInstants = metaClient.reloadActiveTimeline().getInstants.iterator().asScala - .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION && filteredPendingClusteringInstants.contains(p.getTimestamp)) + .filter(p => p.getAction == HoodieTimeline.REPLACE_COMMIT_ACTION && filteredPendingClusteringInstants.contains(p.requestedTime)) .toSeq - .sortBy(f => f.getTimestamp) + .sortBy(f => f.requestedTime) .reverse val clusteringPlans = clusteringInstants.map(instant => @@ -182,12 +182,12 @@ class RunClusteringProcedure extends BaseProcedure if (showInvolvedPartitions) { clusteringPlans.map { p => - Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + Row(p.get().getLeft.requestedTime, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), HoodieCLIUtils.extractPartitions(p.get().getRight.getInputGroups.asScala.toSeq)) } } else { clusteringPlans.map { p => - Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), "*") + Row(p.get().getLeft.requestedTime, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), "*") } } } finally { 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 c5deddd977d3..27946c6e7498 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 @@ -87,7 +87,7 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp val pendingCompactionInstants = metaClient.getActiveTimeline.getWriteTimeline.getInstants.iterator().asScala .filter(p => p.getAction == HoodieTimeline.COMPACTION_ACTION) - .map(_.getTimestamp) + .map(_.requestedTime) .toSeq.sortBy(f => f) var (filteredPendingCompactionInstants, operation) = HoodieProcedureUtils.filterPendingInstantsAndGetOperation( @@ -125,15 +125,15 @@ class RunCompactionProcedure extends BaseProcedure with ProcedureBuilder with Sp } val compactionInstants = metaClient.reloadActiveTimeline().getInstantsAsStream.iterator().asScala - .filter(instant => filteredPendingCompactionInstants.contains(instant.getTimestamp)) + .filter(instant => filteredPendingCompactionInstants.contains(instant.requestedTime)) .toSeq - .sortBy(p => p.getTimestamp) + .sortBy(p => p.requestedTime) .reverse compactionInstants.map(instant => - (instant, CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp)) + (instant, CompactionUtils.getCompactionPlan(metaClient, instant.requestedTime)) ).map { case (instant, plan) => - Row(instant.getTimestamp, plan.getOperations.size(), instant.getState.name()) + Row(instant.requestedTime, plan.getOperations.size(), instant.getState.name()) } } finally { if (client != null) { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunRollbackInflightTableServiceProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunRollbackInflightTableServiceProcedure.scala index 029b7546e16d..bea1af7deb90 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunRollbackInflightTableServiceProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/RunRollbackInflightTableServiceProcedure.scala @@ -28,7 +28,6 @@ import org.apache.hudi.table.HoodieSparkTable import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.PredicateHelper -import org.apache.spark.sql.hudi.command.procedures import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util.function.Supplier @@ -73,12 +72,13 @@ class RunRollbackInflightTableServiceProcedure extends BaseProcedure val basePath: String = getBasePath(tableName, tablePath) val metaClient = HoodieTableMetaClient.builder .setConf(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration)).setBasePath(basePath).build - + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator // determine whether the current instant exists and whether it is clustering or compaction var isClustering: Boolean = true var instant: HoodieInstant = null - val pendingCompactionInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, pendingInstant) - val pendingClusteringInstant = ClusteringUtils.getInflightClusteringInstant(pendingInstant, metaClient.getActiveTimeline) + val pendingCompactionInstant = instantGenerator.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, pendingInstant) + val pendingClusteringInstant = ClusteringUtils.getInflightClusteringInstant(pendingInstant, + metaClient.getActiveTimeline, metaClient.getTimelineLayout.getInstantGenerator) val timeline = metaClient.getActiveTimeline.getWriteTimeline if (!timeline.containsInstant(pendingCompactionInstant) && !pendingClusteringInstant.isPresent) { throw new RuntimeException(s"there is no pending instant : [$pendingClusteringInstant | $pendingCompactionInstant]") @@ -100,14 +100,14 @@ class RunRollbackInflightTableServiceProcedure extends BaseProcedure val startTs = System.currentTimeMillis() doRollbackOnInflightInstant(client, instant, isClustering) if (deleteRequestInstantFile) { - val requestInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, instant.getAction, instant.getTimestamp) + val requestInstant = instantGenerator.createNewInstant(HoodieInstant.State.REQUESTED, instant.getAction, instant.requestedTime) metaClient.getActiveTimeline.deleteInstantFileIfExists(requestInstant) } val timeCost = System.currentTimeMillis() - startTs logInfo(s"Finish rollback pending instant: $pendingInstant," + s" time cost: $timeCost ms.") - Seq(Row(instant.getTimestamp, timeCost.toString)) + Seq(Row(instant.requestedTime, timeCost.toString)) } finally { if (client != null) { client.close() diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowArchivedCommitsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowArchivedCommitsProcedure.scala index a8ccca0ead13..3a40f59952bb 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowArchivedCommitsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowArchivedCommitsProcedure.scala @@ -19,9 +19,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.HoodieCommitMetadata -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieDefaultTimeline, HoodieInstant} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout, TimelineUtils} import org.apache.hudi.common.util.StringUtils - import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} @@ -29,7 +28,6 @@ import java.time.ZonedDateTime import java.util import java.util.{Collections, Date} import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure with ProcedureBuilder { @@ -96,7 +94,7 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr val archivedTimeline = metaClient.getArchivedTimeline try { archivedTimeline.loadInstantDetailsInMemory(startTs, endTs) - val timelineRange = archivedTimeline.findInstantsInRange(startTs, endTs) + val timelineRange = archivedTimeline.findInstantsInRange(startTs, endTs).asInstanceOf[HoodieTimeline] if (includeExtraMetadata) { getCommitsWithMetadata(timelineRange, limit) } else { @@ -110,19 +108,19 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr override def build: Procedure = new ShowArchivedCommitsProcedure(includeExtraMetadata) - private def getCommitsWithMetadata(timeline: HoodieDefaultTimeline, + private def getCommitsWithMetadata(timeline: HoodieTimeline, limit: Int): Seq[Row] = { import scala.collection.JavaConverters._ val (rows: util.ArrayList[Row], newCommits: util.ArrayList[HoodieInstant]) = getSortCommits(timeline) - + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) for (partitionWriteStat <- commitMetadata.getPartitionToWriteStats.entrySet.asScala) { for (hoodieWriteStat <- partitionWriteStat.getValue.asScala) { rows.add(Row( - commit.getTimestamp, commit.getCompletionTime, commit.getAction, hoodieWriteStat.getPartitionPath, + commit.requestedTime, commit.getCompletionTime, commit.getAction, hoodieWriteStat.getPartitionPath, hoodieWriteStat.getFileId, hoodieWriteStat.getPrevCommit, hoodieWriteStat.getNumWrites, hoodieWriteStat.getNumInserts, hoodieWriteStat.getNumDeletes, hoodieWriteStat.getNumUpdateWrites, hoodieWriteStat.getTotalWriteErrors, hoodieWriteStat.getTotalLogBlocks, hoodieWriteStat.getTotalCorruptLogBlock, @@ -135,24 +133,25 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr rows.stream().limit(limit).toArray().map(r => r.asInstanceOf[Row]).toList } - private def getSortCommits(timeline: HoodieDefaultTimeline): (util.ArrayList[Row], util.ArrayList[HoodieInstant]) = { + private def getSortCommits(timeline: HoodieTimeline): (util.ArrayList[Row], util.ArrayList[HoodieInstant]) = { val rows = new util.ArrayList[Row] // timeline can be read from multiple files. So sort is needed instead of reversing the collection val commits: util.List[HoodieInstant] = timeline.getCommitsTimeline.filterCompletedInstants .getInstants.toArray().map(instant => instant.asInstanceOf[HoodieInstant]).toList.asJava val newCommits = new util.ArrayList[HoodieInstant](commits) - Collections.sort(newCommits, HoodieInstant.INSTANT_TIME_COMPARATOR.reversed) + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) + Collections.sort(newCommits, layout.getInstantComparator.requestedTimeOrderedComparator.reversed) (rows, newCommits) } - def getCommits(timeline: HoodieDefaultTimeline, + def getCommits(timeline: HoodieTimeline, limit: Int): Seq[Row] = { val (rows: util.ArrayList[Row], newCommits: util.ArrayList[HoodieInstant]) = getSortCommits(timeline) - + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) - rows.add(Row(commit.getTimestamp, commit.getCompletionTime, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + rows.add(Row(commit.requestedTime, commit.getCompletionTime, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, commitMetadata.fetchTotalFilesUpdated, commitMetadata.fetchTotalPartitionsWritten, commitMetadata.fetchTotalRecordsWritten, commitMetadata.fetchTotalUpdateRecordsWritten, commitMetadata.fetchTotalWriteErrors)) @@ -163,7 +162,7 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr def getTimeDaysAgo(numberOfDays: Int): String = { val date = Date.from(ZonedDateTime.now.minusDays(numberOfDays).toInstant) - HoodieActiveTimeline.formatDate(date) + TimelineUtils.formatDate(date) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala index 060d6cb48aab..7eb5ff72f8b5 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowClusteringProcedure.scala @@ -61,7 +61,7 @@ class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with S val clusteringInstants = metaClient.getActiveTimeline.getInstants.iterator().asScala .filter(p => ClusteringUtils.isClusteringOrReplaceCommitAction(p.getAction)) .toSeq - .sortBy(f => f.getTimestamp) + .sortBy(f => f.requestedTime) .reverse .take(limit) @@ -71,12 +71,12 @@ class ShowClusteringProcedure extends BaseProcedure with ProcedureBuilder with S if (showInvolvedPartitions) { clusteringPlans.map { p => - Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + Row(p.get().getLeft.requestedTime, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), HoodieCLIUtils.extractPartitions(p.get().getRight.getInputGroups.asScala.toSeq)) } } else { clusteringPlans.map { p => - Row(p.get().getLeft.getTimestamp, p.get().getRight.getInputGroups.size(), + Row(p.get().getLeft.requestedTime, p.get().getRight.getInputGroups.size(), p.get().getLeft.getState.name(), "*") } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala index 9d2583392ae0..7a08a686e302 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowColumnStatsOverlapProcedure.scala @@ -23,7 +23,7 @@ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.data.HoodieData import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{FileSlice, HoodieRecord} -import org.apache.hudi.common.table.timeline.{HoodieDefaultTimeline, HoodieInstant} +import org.apache.hudi.common.table.timeline.{BaseHoodieTimeline, HoodieInstant} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.util.{Option => HOption} @@ -264,7 +264,7 @@ class ShowColumnStatsOverlapProcedure extends BaseProcedure with ProcedureBuilde val timeline = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants() val maxInstant = metaClient.createNewInstantTime() - val instants = timeline.getInstants.iterator().asScala.filter(_.getTimestamp < maxInstant) + val instants = timeline.getInstants.iterator().asScala.filter(_.requestedTime < maxInstant) val details = new Function[HoodieInstant, org.apache.hudi.common.util.Option[Array[Byte]]] with java.io.Serializable { @@ -273,7 +273,7 @@ class ShowColumnStatsOverlapProcedure extends BaseProcedure with ProcedureBuilde } } - val filteredTimeline = new HoodieDefaultTimeline( + val filteredTimeline = metaClient.getTimelineLayout.getTimelineFactory.createDefaultTimeline( new java.util.ArrayList[HoodieInstant](instants.toList.asJava).stream(), details) new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitExtraMetadataProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitExtraMetadataProcedure.scala index 1fa4d5669175..addbf564573d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitExtraMetadataProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitExtraMetadataProcedure.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieReplaceCommitMetadata} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.ClusteringUtils import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.Row @@ -65,7 +66,7 @@ class ShowCommitExtraMetadataProcedure() extends BaseProcedure with ProcedureBui val hoodieInstantOption: Option[HoodieInstant] = if (instantTime.isEmpty) { getCommitForLastInstant(timeline) } else { - getCommitForInstant(timeline, instantTime.get.asInstanceOf[String]) + getCommitForInstant(metaClient, timeline, instantTime.get.asInstanceOf[String]) } if (hoodieInstantOption.isEmpty) { @@ -79,7 +80,7 @@ class ShowCommitExtraMetadataProcedure() extends BaseProcedure with ProcedureBui } val meta = commitMetadataOptional.get - val timestamp: String = hoodieInstantOption.get.getTimestamp + val timestamp: String = hoodieInstantOption.get.requestedTime val action: String = hoodieInstantOption.get.getAction val metadatas: util.Map[String, String] = if (metadataKey.isEmpty) { meta.getExtraMetadata @@ -104,24 +105,26 @@ class ShowCommitExtraMetadataProcedure() extends BaseProcedure with ProcedureBui } } - private def getCommitForInstant(timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + private def getCommitForInstant(metaClient: HoodieTableMetaClient, timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val instants: util.List[HoodieInstant] = util.Arrays.asList( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.CLUSTERING_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLUSTERING_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) val hoodieInstant: Option[HoodieInstant] = instants.asScala.find((i: HoodieInstant) => timeline.containsInstant(i)) hoodieInstant } private def getHoodieCommitMetadata(timeline: HoodieTimeline, hoodieInstant: Option[HoodieInstant]): Option[HoodieCommitMetadata] = { + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieReplaceCommitMetadata])) } else { - Option(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieCommitMetadata])) } } else { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitFilesProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitFilesProcedure.scala index f2ecbbc7a9e7..67fef62d3d69 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitFilesProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitFilesProcedure.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieReplaceCommitMetadata, HoodieWriteStat} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.ClusteringUtils import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.Row @@ -28,7 +29,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util import java.util.List import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowCommitFilesProcedure() extends BaseProcedure with ProcedureBuilder { @@ -66,7 +66,7 @@ class ShowCommitFilesProcedure() extends BaseProcedure with ProcedureBuilder { val metaClient = createMetaClient(jsc, basePath) val activeTimeline = metaClient.getActiveTimeline val timeline = activeTimeline.getCommitsTimeline.filterCompletedInstants - val hoodieInstantOption = getCommitForInstant(timeline, instantTime) + val hoodieInstantOption = getCommitForInstant(metaClient, timeline, instantTime) val commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption) if (commitMetadataOptional.isEmpty) { @@ -89,24 +89,26 @@ class ShowCommitFilesProcedure() extends BaseProcedure with ProcedureBuilder { override def build: Procedure = new ShowCommitFilesProcedure() - private def getCommitForInstant(timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + private def getCommitForInstant(metaClient: HoodieTableMetaClient, timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val instants: util.List[HoodieInstant] = util.Arrays.asList( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.CLUSTERING_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLUSTERING_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) val hoodieInstant: Option[HoodieInstant] = instants.asScala.find((i: HoodieInstant) => timeline.containsInstant(i)) hoodieInstant } private def getHoodieCommitMetadata(timeline: HoodieTimeline, hoodieInstant: Option[HoodieInstant]): Option[HoodieCommitMetadata] = { + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieReplaceCommitMetadata])) } else { - Option(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieCommitMetadata])) } } else { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitPartitionsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitPartitionsProcedure.scala index 307ddabef99c..aa758dbdaf1c 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitPartitionsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitPartitionsProcedure.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieReplaceCommitMetadata, HoodieWriteStat} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.ClusteringUtils import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.Row @@ -28,7 +29,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util import java.util.List import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowCommitPartitionsProcedure() extends BaseProcedure with ProcedureBuilder { @@ -65,7 +65,7 @@ class ShowCommitPartitionsProcedure() extends BaseProcedure with ProcedureBuilde val metaClient = createMetaClient(jsc, basePath) val activeTimeline = metaClient.getActiveTimeline val timeline = activeTimeline.getCommitsTimeline.filterCompletedInstants - val hoodieInstantOption = getCommitForInstant(timeline, instantTime) + val hoodieInstantOption = getCommitForInstant(metaClient, timeline, instantTime) val commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption) if (commitMetadataOptional.isEmpty) { @@ -104,12 +104,13 @@ class ShowCommitPartitionsProcedure() extends BaseProcedure with ProcedureBuilde override def build: Procedure = new ShowCommitPartitionsProcedure() - private def getCommitForInstant(timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + private def getCommitForInstant(metaClient: HoodieTableMetaClient, timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val instants: util.List[HoodieInstant] = util.Arrays.asList( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.CLUSTERING_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLUSTERING_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) val hoodieInstant: Option[HoodieInstant] = instants.asScala.find((i: HoodieInstant) => timeline.containsInstant(i)) hoodieInstant @@ -121,7 +122,8 @@ class ShowCommitPartitionsProcedure() extends BaseProcedure with ProcedureBuilde Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieReplaceCommitMetadata])) } else { - Option(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieCommitMetadata])) } } else { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitWriteStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitWriteStatsProcedure.scala index 1bc17803d633..b17b91a6d86d 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitWriteStatsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitWriteStatsProcedure.scala @@ -19,7 +19,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieReplaceCommitMetadata} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.ClusteringUtils import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.Row @@ -27,7 +28,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilder { @@ -60,7 +60,7 @@ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilde val metaClient = createMetaClient(jsc, basePath) val activeTimeline = metaClient.getActiveTimeline val timeline = activeTimeline.getCommitsTimeline.filterCompletedInstants - val hoodieInstantOption = getCommitForInstant(timeline, instantTime) + val hoodieInstantOption = getCommitForInstant(metaClient, timeline, instantTime) val commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption) if (commitMetadataOptional.isEmpty) { @@ -81,12 +81,13 @@ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilde override def build: Procedure = new ShowCommitWriteStatsProcedure() - private def getCommitForInstant(timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + private def getCommitForInstant(metaClient: HoodieTableMetaClient, timeline: HoodieTimeline, instantTime: String): Option[HoodieInstant] = { + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val instants: util.List[HoodieInstant] = util.Arrays.asList( - new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.CLUSTERING_ACTION, instantTime), - new HoodieInstant(false, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.CLUSTERING_ACTION, instantTime), + instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, instantTime)) val hoodieInstant: Option[HoodieInstant] = instants.asScala.find((i: HoodieInstant) => timeline.containsInstant(i)) hoodieInstant @@ -98,7 +99,8 @@ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilde Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieReplaceCommitMetadata])) } else { - Option(HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, classOf[HoodieCommitMetadata])) } } else { diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala index 999e1f9a674e..b9a130d8eb1b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCommitsProcedure.scala @@ -19,15 +19,13 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.model.HoodieCommitMetadata -import org.apache.hudi.common.table.timeline.{HoodieDefaultTimeline, HoodieInstant} - +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util import java.util.Collections import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure with ProcedureBuilder { @@ -95,19 +93,19 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure override def build: Procedure = new ShowCommitsProcedure(includeExtraMetadata) - private def getCommitsWithMetadata(timeline: HoodieDefaultTimeline, + private def getCommitsWithMetadata(timeline: HoodieTimeline, limit: Int): Seq[Row] = { import scala.collection.JavaConverters._ val (rows: util.ArrayList[Row], newCommits: util.ArrayList[HoodieInstant]) = getSortCommits(timeline) - + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) for (partitionWriteStat <- commitMetadata.getPartitionToWriteStats.entrySet.asScala) { for (hoodieWriteStat <- partitionWriteStat.getValue.asScala) { rows.add(Row( - commit.getTimestamp, commit.getCompletionTime, commit.getAction, hoodieWriteStat.getPartitionPath, + commit.requestedTime, commit.getCompletionTime, commit.getAction, hoodieWriteStat.getPartitionPath, hoodieWriteStat.getFileId, hoodieWriteStat.getPrevCommit, hoodieWriteStat.getNumWrites, hoodieWriteStat.getNumInserts, hoodieWriteStat.getNumDeletes, hoodieWriteStat.getNumUpdateWrites, hoodieWriteStat.getTotalWriteErrors, hoodieWriteStat.getTotalLogBlocks, hoodieWriteStat.getTotalCorruptLogBlock, @@ -120,24 +118,25 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure rows.stream().limit(limit).toArray().map(r => r.asInstanceOf[Row]).toList } - private def getSortCommits(timeline: HoodieDefaultTimeline): (util.ArrayList[Row], util.ArrayList[HoodieInstant]) = { + private def getSortCommits(timeline: HoodieTimeline): (util.ArrayList[Row], util.ArrayList[HoodieInstant]) = { val rows = new util.ArrayList[Row] // timeline can be read from multiple files. So sort is needed instead of reversing the collection val commits: util.List[HoodieInstant] = timeline.getCommitsTimeline.filterCompletedInstants .getInstants.toArray().map(instant => instant.asInstanceOf[HoodieInstant]).toList.asJava val newCommits = new util.ArrayList[HoodieInstant](commits) - Collections.sort(newCommits, HoodieInstant.INSTANT_TIME_COMPARATOR.reversed) + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) + Collections.sort(newCommits, layout.getInstantComparator.requestedTimeOrderedComparator.reversed) (rows, newCommits) } - def getCommits(timeline: HoodieDefaultTimeline, + def getCommits(timeline: HoodieTimeline, limit: Int): Seq[Row] = { val (rows: util.ArrayList[Row], newCommits: util.ArrayList[HoodieInstant]) = getSortCommits(timeline) - + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) - rows.add(Row(commit.getTimestamp, commit.getCompletionTime, commit.getAction, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + rows.add(Row(commit.requestedTime, commit.getCompletionTime, commit.getAction, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, commitMetadata.fetchTotalFilesUpdated, commitMetadata.fetchTotalPartitionsWritten, commitMetadata.fetchTotalRecordsWritten, commitMetadata.fetchTotalUpdateRecordsWritten, commitMetadata.fetchTotalWriteErrors)) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala index 2fb629d56f64..a25ceeb4cd14 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowCompactionProcedure.scala @@ -66,14 +66,14 @@ class ShowCompactionProcedure extends BaseProcedure with ProcedureBuilder with S val compactionInstants = metaClient.getActiveTimeline.getInstants.iterator().asScala .filter(p => p.getAction == HoodieTimeline.COMPACTION_ACTION || p.getAction == HoodieTimeline.COMMIT_ACTION) .toSeq - .sortBy(f => f.getTimestamp) + .sortBy(f => f.requestedTime) .reverse .take(limit) compactionInstants.map(instant => - (instant, CompactionUtils.getCompactionPlan(metaClient, instant.getTimestamp)) + (instant, CompactionUtils.getCompactionPlan(metaClient, instant.requestedTime)) ).map { case (instant, plan) => - Row(instant.getTimestamp, plan.getOperations.size(), instant.getState.name()) + Row(instant.requestedTime, plan.getOperations.size(), instant.getState.name()) } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileStatusProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileStatusProcedure.scala index 1eaa810d7a28..bc9ded08c1ae 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileStatusProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileStatusProcedure.scala @@ -21,7 +21,7 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hadoop.fs.Path import org.apache.hudi.HoodieCLIUtils import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieArchivedTimeline, HoodieDefaultTimeline, HoodieInstant, HoodieTimeline, TimelineMetadataUtils} +import org.apache.hudi.common.table.timeline.{HoodieTimeline, HoodieActiveTimeline, HoodieArchivedTimeline, HoodieInstant, TimelineMetadataUtils} import org.apache.hudi.exception.HoodieException import org.apache.hudi.table.HoodieSparkTable import org.apache.spark.internal.Logging @@ -140,7 +140,7 @@ class ShowFileStatusProcedure extends BaseProcedure FileStatusInfo( FileStatus.DELETED.toString, HoodieTimeline.RESTORE_ACTION, - restoreInstant.getTimestamp, + restoreInstant.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE ) @@ -154,7 +154,7 @@ class ShowFileStatusProcedure extends BaseProcedure ) } - private def checkRollbackMetadataInternal(timeline: HoodieDefaultTimeline, + private def checkRollbackMetadataInternal(timeline: HoodieTimeline, partition: Option[String], fileName: String): Option[FileStatusInfo] = { val rollbackInstant = timeline.getRollbackTimeline .filterCompletedInstants() @@ -172,7 +172,7 @@ class ShowFileStatusProcedure extends BaseProcedure p => Option.apply(partitionRollbackMetadata.get(p)).flatMap( _.getSuccessDeleteFiles.asScala.find(_.contains(fileName)))).isDefined || partitionRollbackMetadata.values.iterator.asScala.exists(_.getSuccessDeleteFiles.asScala.exists(_.contains(fileName))) - }.map(instant => getResult(timeline, HoodieTimeline.ROLLBACK_ACTION, instant.getTimestamp).get) + }.map(instant => getResult(timeline, HoodieTimeline.ROLLBACK_ACTION, instant.requestedTime).get) } @@ -181,7 +181,7 @@ class ShowFileStatusProcedure extends BaseProcedure .orElse(checkCleanMetadataInternal(metaClient.getArchivedTimeline, partition, fileName)) } - private def checkCleanMetadataInternal(timeline: HoodieDefaultTimeline, partition: Option[String], fileName: String): Option[FileStatusInfo] = { + private def checkCleanMetadataInternal(timeline: HoodieTimeline, partition: Option[String], fileName: String): Option[FileStatusInfo] = { val cleanedInstant = timeline.getCleanerTimeline .filterCompletedInstants() .getReverseOrderedInstants @@ -194,10 +194,10 @@ class ShowFileStatusProcedure extends BaseProcedure val partitionCleanMetadata = cleanMetadata.getPartitionMetadata partition.flatMap(p => Option.apply(partitionCleanMetadata.get(p)).flatMap(_.getSuccessDeleteFiles.asScala.find(_.contains(fileName)))).isDefined || partitionCleanMetadata.values.iterator.asScala.exists(_.getSuccessDeleteFiles.asScala.exists(_.contains(fileName))) - }.map(instant => getResult(timeline, HoodieTimeline.CLEAN_ACTION, instant.getTimestamp).get) + }.map(instant => getResult(timeline, HoodieTimeline.CLEAN_ACTION, instant.requestedTime).get) } - private def getResult(timeline: HoodieDefaultTimeline, action: String, timestamp: String): Option[FileStatusInfo] = { + private def getResult(timeline: HoodieTimeline, action: String, timestamp: String): Option[FileStatusInfo] = { timeline match { case _: HoodieActiveTimeline => Option.apply(FileStatusInfo(FileStatus.DELETED.toString, action, timestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) @@ -207,7 +207,7 @@ class ShowFileStatusProcedure extends BaseProcedure } } - private def reloadTimelineIfNecessary(timeline: HoodieDefaultTimeline): Unit = { + private def reloadTimelineIfNecessary(timeline: HoodieTimeline): Unit = { timeline match { case _: HoodieArchivedTimeline => val archivalTimeline: HoodieArchivedTimeline = timeline.asInstanceOf[HoodieArchivedTimeline] diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala index aa4e11304ebf..a61358bdd46b 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowFileSystemViewProcedure.scala @@ -20,19 +20,17 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{FileSlice, HoodieLogFile} import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.{CompletionTimeQueryView, HoodieDefaultTimeline, HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, InstantComparison} import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.util import org.apache.hudi.exception.HoodieException import org.apache.hudi.storage.StoragePath - import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util.function.{Function, Supplier} import java.util.stream.{Collectors, Stream => JStream} import java.util.{ArrayList => JArrayList, List => JList} - import scala.collection.JavaConverters._ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure with ProcedureBuilder { @@ -109,11 +107,11 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit var instants = timeline.getInstants.iterator().asScala if (maxInstant.nonEmpty) { val predicate = if (includeMaxInstant) { - HoodieTimeline.GREATER_THAN_OR_EQUALS + InstantComparison.GREATER_THAN_OR_EQUALS } else { - HoodieTimeline.GREATER_THAN + InstantComparison.GREATER_THAN } - instants = instants.filter(instant => predicate.test(maxInstant, instant.getTimestamp)) + instants = instants.filter(instant => predicate.test(maxInstant, instant.requestedTime)) } val details = new Function[HoodieInstant, org.apache.hudi.common.util.Option[Array[Byte]]] @@ -123,7 +121,7 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit } } - val filteredTimeline = new HoodieDefaultTimeline( + val filteredTimeline = metaClient.getTimelineLayout.getTimelineFactory.createDefaultTimeline( new JArrayList[HoodieInstant](instants.toList.asJava).stream(), details) new HoodieTableFileSystemView(metaClient, filteredTimeline, statuses) } @@ -156,7 +154,7 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit maxInstant: String, merge: Boolean): JList[Row] = { var fileSliceStream: JStream[FileSlice] = JStream.empty() - val completionTimeQueryView = new CompletionTimeQueryView(metaClient) + val completionTimeQueryView =metaClient.getTimelineLayout().getTimelineFactory().createCompletionTimeQueryView(metaClient) if (merge) { partitions.foreach(p => fileSliceStream = JStream.concat(fileSliceStream, fsView.getLatestMergedFileSlicesBeforeOrOn(p, maxInstant))) } else { @@ -250,7 +248,7 @@ class ShowFileSystemViewProcedure(showLatest: Boolean) extends BaseProcedure wit val maxInstantForMerge = if (merge && maxInstant.isEmpty) { val lastInstant = metaClient.getActiveTimeline.filterCompletedAndCompactionInstants().lastInstant() if (lastInstant.isPresent) { - lastInstant.get().getTimestamp + lastInstant.get().requestedTime } else { // scalastyle:off return return Seq.empty diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala index 41a8fd433fcf..7d2de35fb409 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowHoodieLogFileRecordsProcedure.scala @@ -68,7 +68,7 @@ class ShowHoodieLogFileRecordsProcedure extends BaseProcedure with ProcedureBuil .withBasePath(basePath) .withLogFilePaths(logFilePaths.asJava) .withReaderSchema(schema) - .withLatestInstantTime(client.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.getTimestamp) + .withLatestInstantTime(client.getActiveTimeline.getCommitAndReplaceTimeline.lastInstant.get.requestedTime) .withReverseReader(java.lang.Boolean.parseBoolean(HoodieReaderConfig.COMPACTION_REVERSE_LOG_READ_ENABLE.defaultValue)) .withBufferSize(HoodieMemoryConfig.MAX_DFS_STREAM_BUFFER_SIZE.defaultValue) .withMaxMemorySizeInBytes(HoodieMemoryConfig.DEFAULT_MAX_MEMORY_FOR_SPILLABLE_MAP_IN_BYTES) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala index c60b2538bb61..276ccf191102 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowMetadataTableColumnStatsProcedure.scala @@ -23,14 +23,13 @@ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.data.HoodieData import org.apache.hudi.common.model.FileSlice import org.apache.hudi.common.table.TableSchemaResolver -import org.apache.hudi.common.table.timeline.{HoodieDefaultTimeline, HoodieInstant} +import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.exception.HoodieException import org.apache.hudi.metadata.HoodieTableMetadata import org.apache.hudi.storage.StoragePathInfo import org.apache.hudi.{AvroConversionUtils, ColumnStatsIndexSupport} - import org.apache.avro.generic.IndexedRecord import org.apache.spark.internal.Logging import org.apache.spark.sql.Row @@ -38,7 +37,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.util import java.util.function.{Function, Supplier} - import scala.collection.JavaConverters._ import scala.collection.mutable @@ -139,7 +137,7 @@ class ShowMetadataTableColumnStatsProcedure extends BaseProcedure with Procedure val timeline = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants() val maxInstant = metaClient.createNewInstantTime() - val instants = timeline.getInstants.iterator().asScala.filter(_.getTimestamp < maxInstant) + val instants = timeline.getInstants.iterator().asScala.filter(_.requestedTime < maxInstant) val details = new Function[HoodieInstant, org.apache.hudi.common.util.Option[Array[Byte]]] with java.io.Serializable { @@ -148,7 +146,7 @@ class ShowMetadataTableColumnStatsProcedure extends BaseProcedure with Procedure } } - val filteredTimeline = new HoodieDefaultTimeline( + val filteredTimeline = metaClient.getTimelineLayout.getTimelineFactory.createDefaultTimeline( new java.util.ArrayList[HoodieInstant](instants.toList.asJava).stream(), details) new HoodieTableFileSystemView(metaClient, filteredTimeline, new java.util.ArrayList[StoragePathInfo]) diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowRollbacksProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowRollbacksProcedure.scala index 2588f82b78c4..ad9e4d08e695 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowRollbacksProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowRollbacksProcedure.scala @@ -18,18 +18,17 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.avro.model.HoodieRollbackMetadata +import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.HoodieTimeline.ROLLBACK_ACTION import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, HoodieInstant, TimelineMetadataUtils} import org.apache.hudi.exception.HoodieException - import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.io.IOException import java.util import java.util.function.Supplier - import scala.collection.JavaConverters._ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with ProcedureBuilder { @@ -75,7 +74,7 @@ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with Pr val activeTimeline = metaClient.getActiveTimeline if (showDetails) { val instantTime = getArgValueOrDefault(args, parameters(2)).get.asInstanceOf[String] - getRollbackDetail(activeTimeline, instantTime, limit) + getRollbackDetail(metaClient, activeTimeline, instantTime, limit) } else { getRollbacks(activeTimeline, limit) } @@ -83,12 +82,14 @@ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with Pr override def build: Procedure = new ShowRollbacksProcedure(showDetails) - def getRollbackDetail(activeTimeline: HoodieActiveTimeline, + def getRollbackDetail(metaClient: HoodieTableMetaClient, + activeTimeline: HoodieActiveTimeline, instantTime: String, limit: Int): Seq[Row] = { val rows = new util.ArrayList[Row] + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator val metadata = TimelineMetadataUtils.deserializeAvroMetadata(activeTimeline.getInstantDetails( - new HoodieInstant(State.COMPLETED, ROLLBACK_ACTION, instantTime)).get, classOf[HoodieRollbackMetadata]) + instantGenerator.createNewInstant(State.COMPLETED, ROLLBACK_ACTION, instantTime)).get, classOf[HoodieRollbackMetadata]) metadata.getPartitionMetadata.asScala.toMap.iterator.foreach(entry => Stream .concat(entry._2.getSuccessDeleteFiles.asScala.map(f => (f, true)), @@ -117,7 +118,7 @@ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with Pr }) } catch { case e: IOException => - throw new HoodieException(s"Failed to get rollback's info from instant ${instant.getTimestamp}.") + throw new HoodieException(s"Failed to get rollback's info from instant ${instant.requestedTime}.") } }) rows.stream().limit(limit).toArray().map(r => r.asInstanceOf[Row]).toList diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala index 15c808933698..5a2e99795316 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/ShowSavepointsProcedure.scala @@ -54,7 +54,7 @@ class ShowSavepointsProcedure extends BaseProcedure with ProcedureBuilder { val commits: util.List[HoodieInstant] = timeline.getReverseOrderedInstants.collect(Collectors.toList[HoodieInstant]) if (commits.isEmpty) Seq.empty[Row] else { - commits.toArray.map(instant => instant.asInstanceOf[HoodieInstant].getTimestamp).map(p => Row(p)).toSeq + commits.toArray.map(instant => instant.asInstanceOf[HoodieInstant].requestedTime).map(p => Row(p)).toSeq } } diff --git a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/StatsWriteAmplificationProcedure.scala b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/StatsWriteAmplificationProcedure.scala index 433687fb0502..3a4d6184b73e 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/StatsWriteAmplificationProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/main/scala/org/apache/spark/sql/hudi/command/procedures/StatsWriteAmplificationProcedure.scala @@ -18,13 +18,12 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.common.model.HoodieCommitMetadata - +import org.apache.hudi.common.table.timeline.TimelineLayout import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} import java.text.DecimalFormat import java.util.function.Supplier - import scala.collection.JavaConverters.asScalaIteratorConverter class StatsWriteAmplificationProcedure extends BaseProcedure with ProcedureBuilder { @@ -53,14 +52,15 @@ class StatsWriteAmplificationProcedure extends BaseProcedure with ProcedureBuild val df = new DecimalFormat("#.00") var totalRecordsUpserted = 0L var totalRecordsWritten = 0L + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) timeline.getInstants.iterator.asScala.foreach( instantTime => { var waf = "0" - val commit = HoodieCommitMetadata.fromBytes(activeTimeline.getInstantDetails(instantTime).get(), classOf[HoodieCommitMetadata]) + val commit = layout.getCommitMetadataSerDe.deserialize(instantTime, activeTimeline.getInstantDetails(instantTime).get(), classOf[HoodieCommitMetadata]) if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format(commit.fetchTotalRecordsWritten().toFloat / commit.fetchTotalUpdateRecordsWritten()) } - rows.add(Row(instantTime.getTimestamp, commit.fetchTotalUpdateRecordsWritten, commit.fetchTotalRecordsWritten, waf)) + rows.add(Row(instantTime.requestedTime, commit.fetchTotalUpdateRecordsWritten, commit.fetchTotalRecordsWritten, waf)) totalRecordsUpserted = totalRecordsUpserted + commit.fetchTotalUpdateRecordsWritten() totalRecordsWritten = totalRecordsWritten + commit.fetchTotalRecordsWritten() } 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 05192dbb051c..80f993d37fa3 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 @@ -20,8 +20,8 @@ package org.apache.spark.sql.hudi.command.procedures import org.apache.hudi.common.model.HoodieCommitMetadata import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} - +import org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, InstantComparison} import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} @@ -30,7 +30,6 @@ import org.joda.time.DateTime import java.io.IOException import java.sql.{Connection, DriverManager, ResultSet, SQLException} import java.util.function.Supplier - import scala.collection.JavaConverters._ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder with Logging { @@ -100,12 +99,12 @@ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder wi } val targetLatestCommit = - if (targetTimeline.getInstants.iterator().hasNext) targetTimeline.lastInstant().get().getTimestamp else "0" + if (targetTimeline.getInstants.iterator().hasNext) targetTimeline.lastInstant().get().requestedTime else "0" val sourceLatestCommit = - if (sourceTimeline.getInstants.iterator().hasNext) sourceTimeline.lastInstant().get().getTimestamp else "0" + if (sourceTimeline.getInstants.iterator().hasNext) sourceTimeline.lastInstant().get().requestedTime else "0" if (sourceLatestCommit != null - && HoodieTimeline.compareTimestamps(targetLatestCommit, HoodieTimeline.GREATER_THAN, sourceLatestCommit)) + && compareTimestamps(targetLatestCommit, InstantComparison.GREATER_THAN, sourceLatestCommit)) Seq(Row(getString(targetMetaClient, targetTimeline, srcMetaClient, sourceCount, targetCount, sourceLatestCommit))) else Seq(Row(getString(srcMetaClient, sourceTimeline, targetMetaClient, targetCount, sourceCount, targetLatestCommit))) @@ -119,7 +118,7 @@ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder wi if (commitsToCatchup.isEmpty) { s"Count difference now is count(${target.getTableConfig.getTableName}) - count(${source.getTableConfig.getTableName}) == ${targetCount - sourceCount}" } else { - val newInserts = countNewRecords(target, commitsToCatchup.map(elem => elem.getTimestamp)) + val newInserts = countNewRecords(target, commitsToCatchup.map(elem => elem.requestedTime)) s"Count difference now is count(${target.getTableConfig.getTableName}) - count(${source.getTableConfig.getTableName}) == ${targetCount - sourceCount}" + s". Catach up count is $newInserts" } @@ -193,7 +192,10 @@ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder wi var totalNew: Long = 0 val timeline: HoodieTimeline = target.reloadActiveTimeline.getCommitAndReplaceTimeline.filterCompletedInstants for (commit <- commitsToCatchup) { - val c: HoodieCommitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, commit)).get, classOf[HoodieCommitMetadata]) + val instantGenerator = target.getTimelineLayout.getInstantGenerator + val instant: HoodieInstant = instantGenerator.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, commit) + val c: HoodieCommitMetadata = target.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, timeline.getInstantDetails(instant).get, + classOf[HoodieCommitMetadata]) totalNew += c.fetchTotalRecordsWritten - c.fetchTotalUpdateRecordsWritten } totalNew diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java index 1a4bc5af84dd..c841e7734f82 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestHoodieMergeHandleWithSparkMerger.java @@ -270,7 +270,7 @@ public void insertAndUpdate(HoodieWriteConfig writeConfig, int expectedRecordNum // Check metadata files. Option deltaCommit = reloadedMetaClient.getActiveTimeline().getDeltaCommitTimeline().lastInstant(); assertTrue(deltaCommit.isPresent()); - assertEquals(instantTime, deltaCommit.get().getTimestamp(), "Delta commit should be specified value"); + assertEquals(instantTime, deltaCommit.get().requestedTime(), "Delta commit should be specified value"); // Check data files. List fileIds = getFileIds(getPartitionPath()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestMetadataTableSupport.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestMetadataTableSupport.java index 905a67cea796..a6a17fb7a685 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestMetadataTableSupport.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/TestMetadataTableSupport.java @@ -83,11 +83,11 @@ void testRecreateMDTForInsertOverwriteTableOperation() { List instants = timeline.getInstants(); assertEquals(3, instants.size()); // For MDT bootstrap instant. - assertEquals("00000000000000000", instants.get(0).getTimestamp()); + assertEquals("00000000000000000", instants.get(0).requestedTime()); // For RLI bootstrap instant. - assertEquals("00000000000000001", instants.get(1).getTimestamp()); + assertEquals("00000000000000001", instants.get(1).requestedTime()); // For the insert instant. - assertEquals(timestamp0, instants.get(2).getTimestamp()); + assertEquals(timestamp0, instants.get(2).requestedTime()); // Insert second batch. String timestamp1 = "20241015000000001"; @@ -103,11 +103,11 @@ void testRecreateMDTForInsertOverwriteTableOperation() { instants = timeline.getInstants(); assertEquals(3, timeline.getInstants().size()); // For MDT bootstrap instant. - assertEquals("00000000000000000", instants.get(0).getTimestamp()); + assertEquals("00000000000000000", instants.get(0).requestedTime()); // For RLI bootstrap instant. - assertEquals("00000000000000001", instants.get(1).getTimestamp()); + assertEquals("00000000000000001", instants.get(1).requestedTime()); // For the insert_overwrite_table instant. - assertEquals(timestamp1, instants.get(2).getTimestamp()); + assertEquals(timestamp1, instants.get(2).requestedTime()); } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java index 6969357763e7..352aa5a0d476 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestBootstrap.java @@ -35,10 +35,9 @@ import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieTableType; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.RawTripTestPayload; @@ -106,6 +105,8 @@ import java.util.stream.IntStream; import java.util.stream.StreamSupport; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static java.util.stream.Collectors.mapping; import static java.util.stream.Collectors.toList; import static org.apache.spark.sql.functions.callUDF; @@ -276,11 +277,11 @@ private void testBootstrapCommon(boolean partitioned, boolean deltaCommit, Effec // Rollback Bootstrap metaClient.getActiveTimeline().reload().getInstantsAsStream() - .filter(s -> s.equals(new HoodieInstant(State.COMPLETED, + .filter(s -> s.equals(INSTANT_GENERATOR.createNewInstant(State.COMPLETED, deltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION, bootstrapCommitInstantTs))) - .forEach(instant -> HoodieActiveTimeline.deleteInstantFile( - metaClient.getStorage(), metaClient.getMetaPath(), instant)); + .forEach(instant -> TimelineUtils.deleteInstantFile( + metaClient.getStorage(), metaClient.getMetaPath(), instant, INSTANT_FILE_NAME_GENERATOR)); metaClient.reloadActiveTimeline(); client.getTableServiceClient().rollbackFailedBootstrap(); metaClient.reloadActiveTimeline(); @@ -379,7 +380,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta metaClient.reloadActiveTimeline(); assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); assertEquals(instant, metaClient.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); + .getCommitsTimeline().filterCompletedInstants().lastInstant().get().requestedTime()); verifyNoMarkerInTempFolder(); Dataset bootstrapped = sqlContext.read().format("parquet").load(basePath); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java index a963a1d70a3a..68aa635d70db 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestDataSkippingWithMORColstats.java @@ -55,6 +55,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.testutils.RawTripTestPayload.recordToString; import static org.apache.hudi.config.HoodieCompactionConfig.INLINE_COMPACT_NUM_DELTA_COMMITS; import static org.apache.spark.sql.SaveMode.Append; @@ -475,7 +476,7 @@ protected Dataset makeInsertDf(String instantTime, Integer n) { } public void deleteLatestDeltacommit() { - String filename = metaClient.getActiveTimeline().lastInstant().get().getFileName(); + String filename = INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.getActiveTimeline().lastInstant().get()); File deltacommit = new File(metaClient.getBasePath() + "/.hoodie/" + filename); deltacommit.delete(); } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestGlobalIndexEnableUpdatePartitions.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestGlobalIndexEnableUpdatePartitions.java index aa1071fc5c8e..a53db3b6c3cc 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestGlobalIndexEnableUpdatePartitions.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestGlobalIndexEnableUpdatePartitions.java @@ -27,9 +27,9 @@ import org.apache.hudi.common.model.HoodieRecordMerger; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieIndexConfig; import org.apache.hudi.config.HoodiePayloadConfig; @@ -63,6 +63,7 @@ import static org.apache.hudi.common.testutils.HoodieAdaptablePayloadDataGenerator.getPayloadProps; import static org.apache.hudi.common.testutils.HoodieAdaptablePayloadDataGenerator.getUpdates; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.getCommitTimeAtUTC; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_BLOOM; import static org.apache.hudi.index.HoodieIndex.IndexType.GLOBAL_SIMPLE; import static org.apache.hudi.index.HoodieIndex.IndexType.RECORD_INDEX; @@ -170,12 +171,12 @@ public void testRollbacksWithPartitionUpdate(HoodieTableType tableType, IndexTyp try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig)) { // 1st batch: inserts - String commitTimeAtEpoch0 = HoodieActiveTimeline.createNewInstantTime(false, timeGenerator); + String commitTimeAtEpoch0 = TimelineUtils.generateInstantTime(false, timeGenerator); client.startCommitWithTime(commitTimeAtEpoch0); assertNoWriteErrors(client.upsert(jsc().parallelize(insertsAtEpoch0, 2), commitTimeAtEpoch0).collect()); // 2nd batch: update 4 records from p1 to p2 - String commitTimeAtEpoch5 = HoodieActiveTimeline.createNewInstantTime(false, timeGenerator); + String commitTimeAtEpoch5 = TimelineUtils.generateInstantTime(false, timeGenerator); client.startCommitWithTime(commitTimeAtEpoch5); if (isUpsert) { assertNoWriteErrors(client.upsert(jsc().parallelize(updatesAtEpoch5, 2), commitTimeAtEpoch5).collect()); @@ -187,13 +188,13 @@ public void testRollbacksWithPartitionUpdate(HoodieTableType tableType, IndexTyp readTableAndValidate(metaClient, new int[] {}, p2, 0); } // simuate crash. delete latest completed dc. - String latestCompletedDeltaCommit = metaClient.reloadActiveTimeline().getCommitsAndCompactionTimeline().lastInstant().get().getFileName(); + String latestCompletedDeltaCommit = INSTANT_FILE_NAME_GENERATOR.getFileName(metaClient.reloadActiveTimeline().getCommitsAndCompactionTimeline().lastInstant().get()); metaClient.getStorage().deleteFile(new StoragePath(metaClient.getBasePath() + "/.hoodie/" + latestCompletedDeltaCommit)); } try (SparkRDDWriteClient client = getHoodieWriteClient(writeConfig)) { // re-ingest same batch - String commitTimeAtEpoch10 = HoodieActiveTimeline.createNewInstantTime(false, timeGenerator); + String commitTimeAtEpoch10 = TimelineUtils.generateInstantTime(false, timeGenerator); client.startCommitWithTime(commitTimeAtEpoch10); if (isUpsert) { assertNoWriteErrors(client.upsert(jsc().parallelize(updatesAtEpoch5, 2), commitTimeAtEpoch10).collect()); @@ -210,7 +211,7 @@ public void testRollbacksWithPartitionUpdate(HoodieTableType tableType, IndexTyp // update 4 of them from p2 to p3. // delete test: // update 4 of them to p3. these are treated as new inserts since they are deleted. no changes should be seen wrt p2. - String commitTimeAtEpoch15 = HoodieActiveTimeline.createNewInstantTime(false, timeGenerator); + String commitTimeAtEpoch15 = TimelineUtils.generateInstantTime(false, timeGenerator); List updatesAtEpoch15 = getUpdates(updatesAtEpoch5, p3, 15, payloadClass); client.startCommitWithTime(commitTimeAtEpoch15); assertNoWriteErrors(client.upsert(jsc().parallelize(updatesAtEpoch15, 2), commitTimeAtEpoch15).collect()); @@ -219,7 +220,7 @@ public void testRollbacksWithPartitionUpdate(HoodieTableType tableType, IndexTyp readTableAndValidate(metaClient, new int[] {0, 1, 2, 3}, p3, 15); // lets move 2 of them back to p1 - String commitTimeAtEpoch20 = HoodieActiveTimeline.createNewInstantTime(false, timeGenerator); + String commitTimeAtEpoch20 = TimelineUtils.generateInstantTime(false, timeGenerator); List updatesAtEpoch20 = getUpdates(updatesAtEpoch5.subList(0, 2), p1, 20, payloadClass); client.startCommitWithTime(commitTimeAtEpoch20); assertNoWriteErrors(client.upsert(jsc().parallelize(updatesAtEpoch20, 1), commitTimeAtEpoch20).collect()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java index 67b0b929d6ca..f40f823bba47 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestHoodieSparkMergeOnReadTableClustering.java @@ -253,7 +253,7 @@ private void doClusteringAndValidate(SparkRDDWriteClient client, HoodieTimeline timeline = metaClient.getCommitTimeline().filterCompletedInstants(); assertEquals(1, timeline.findInstantsAfter("003", Integer.MAX_VALUE).countInstants(), "Expecting a single commit."); - assertEquals(clusteringCommitTime, timeline.lastInstant().get().getTimestamp()); + assertEquals(clusteringCommitTime, timeline.lastInstant().get().requestedTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, timeline.lastInstant().get().getAction()); if (cfg.populateMetaFields()) { assertEquals(400, HoodieClientTestUtils.countRecordsOptionallySince(jsc(), basePath(), sqlContext(), timeline, Option.of("000")), diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java index 613e620fc278..2263f1630c9e 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestOrcBootstrap.java @@ -364,7 +364,7 @@ private void checkBootstrapResults(int totalRecords, Schema schema, String insta metaClient.reloadActiveTimeline(); assertEquals(expNumInstants, metaClient.getCommitsTimeline().filterCompletedInstants().countInstants()); assertEquals(instant, metaClient.getActiveTimeline() - .getCommitsTimeline().filterCompletedInstants().lastInstant().get().getTimestamp()); + .getCommitsTimeline().filterCompletedInstants().lastInstant().get().requestedTime()); Dataset bootstrapped = sqlContext.read().format("orc").load(basePath); Dataset original = sqlContext.read().format("orc").load(bootstrapBasePath); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java index ef47725f90f1..7f2663d1051c 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java @@ -24,6 +24,7 @@ import org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy; import org.apache.hudi.client.clustering.update.strategy.SparkConsistentBucketDuplicateUpdateStrategy; import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.versioning.v2.TimelineArchiverV2; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; @@ -181,7 +182,7 @@ public void testLoadMetadata(boolean isCommitFilePresent, boolean rowWriterEnabl metaClient = HoodieTableMetaClient.reload(metaClient); final HoodieTable table = HoodieSparkTable.create(config, context, metaClient); writeClient.clean(); - HoodieTimelineArchiver hoodieTimelineArchiver = new HoodieTimelineArchiver(writeClient.getConfig(), table); + HoodieTimelineArchiver hoodieTimelineArchiver = new TimelineArchiverV2(writeClient.getConfig(), table); hoodieTimelineArchiver.archiveIfRequired(context); Arrays.stream(dataGen.getPartitionPaths()).forEach(p -> { if (!isCommitFilePresent) { diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java index 56b570e8a066..bc7bc5edfc58 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkSortAndSizeClustering.java @@ -26,7 +26,6 @@ import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.ClusteringUtils; @@ -58,6 +57,7 @@ import java.util.Properties; import java.util.stream.Collectors; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; public class TestSparkSortAndSizeClustering extends HoodieSparkClientTestHarness { @@ -118,7 +118,7 @@ public void writeAndClustering(boolean isRow) throws IOException { String clusteringTime = (String) writeClient.scheduleClustering(Option.empty()).get(); HoodieClusteringPlan plan = ClusteringUtils.getClusteringPlan( - metaClient, HoodieTimeline.getClusteringCommitRequestedInstant(clusteringTime)).map(Pair::getRight).get(); + metaClient, INSTANT_GENERATOR.getClusteringCommitRequestedInstant(clusteringTime)).map(Pair::getRight).get(); List inputGroups = plan.getInputGroups(); assertEquals(1, inputGroups.size(), "Clustering plan will contain 1 input group"); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestWriteClient.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestWriteClient.java index e6363eac1a7e..ff83e65b7a54 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestWriteClient.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestWriteClient.java @@ -38,6 +38,7 @@ import org.junit.jupiter.api.Test; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -58,7 +59,7 @@ public void testInertsWithEmptyCommitsHavingWriterSchemaAsNull() throws Exceptio String firstCommit = "001"; int numRecords = 200; JavaRDD result = insertFirstBatch(cfgBuilder.build(), client, firstCommit, "000", numRecords, SparkRDDWriteClient::insert, - false, false, numRecords); + false, false, numRecords, INSTANT_GENERATOR); assertTrue(client.commit(firstCommit, result), "Commit should succeed"); // Re-init client with null writer schema. @@ -73,7 +74,8 @@ public void testInertsWithEmptyCommitsHavingWriterSchemaAsNull() throws Exceptio // Schema Validations. HoodieTableMetaClient metaClient = createMetaClient(jsc, basePath); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); - HoodieCommitMetadata metadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(timeline.lastInstant().get(), + timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); assertTrue(metadata.getExtraMetadata().get("schema").isEmpty()); TableSchemaResolver tableSchemaResolver = new TableSchemaResolver(metaClient); assertEquals(Schema.parse(TRIP_EXAMPLE_SCHEMA), tableSchemaResolver.getTableAvroSchema(false)); diff --git a/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties index d74c0444a572..97087c10a321 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties +++ b/hudi-spark-datasource/hudi-spark/src/test/resources/table-config.properties @@ -18,4 +18,4 @@ hoodie.table.name=test_table hoodie.table.type=COPY_ON_WRITE hoodie.archivelog.folder=archive -hoodie.timeline.layout.version=1 +hoodie.timeline.layout.version=2 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 55db8ac26702..574f458872ab 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -306,7 +306,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS val listFilesAfterFirstWrite = fileIndexFirstWrite.listFiles(Nil, Nil) val distinctListOfCommitTimesAfterFirstWrite = getDistinctCommitTimeFromAllFilesInIndex(listFilesAfterFirstWrite) - val firstWriteCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + val firstWriteCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().requestedTime assertEquals(1, distinctListOfCommitTimesAfterFirstWrite.size, "Should have only one commit") assertEquals(firstWriteCommitTime, distinctListOfCommitTimesAfterFirstWrite.head, "All files should belong to the first existing commit") @@ -326,7 +326,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS val fileSlicesAfterSecondWrite = fileIndexFirstWrite.listFiles(Nil, Nil) val distinctListOfCommitTimesAfterSecondWrite = getDistinctCommitTimeFromAllFilesInIndex(fileSlicesAfterSecondWrite) metaClient = HoodieTableMetaClient.reload(metaClient) - val lastCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + val lastCommitTime = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().requestedTime assertEquals(1, distinctListOfCommitTimesAfterSecondWrite.size, "All basefiles affected so all have same commit time") assertEquals(lastCommitTime, distinctListOfCommitTimesAfterSecondWrite.head, "All files should be of second commit after index refresh") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestTableSchemaResolverWithSparkSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestTableSchemaResolverWithSparkSQL.scala index 938c739c92ea..8f1e2c16c5ea 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestTableSchemaResolverWithSparkSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestTableSchemaResolverWithSparkSQL.scala @@ -72,7 +72,8 @@ class TestTableSchemaResolverWithSparkSQL extends HoodieSparkWriterTestBase { // Delete latest metadata table deltacommit // Get schema from metadata table hfile format base file. val latestInstant = metaClient.getActiveTimeline.getCommitsTimeline.getReverseOrderedInstants.findFirst() - val path = new Path(metadataTablePath + "/.hoodie", latestInstant.get().getFileName) + val instantFileNameGenerator = metaClient.getTimelineLayout.getInstantFileNameGenerator + val path = new Path(metadataTablePath + "/.hoodie", instantFileNameGenerator.getFileName(latestInstant.get())) val fs = path.getFileSystem(new Configuration()) fs.delete(path, false) schemaValuationBasedOnDataFile(metaClient, HoodieMetadataRecord.getClassSchema.toString()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala index 8839f6d55d28..b0e23ac19021 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/ColumnStatIndexTestBase.scala @@ -179,7 +179,7 @@ class ColumnStatIndexTestBase extends HoodieSparkClientTestBase { } else { val colsToGenerateStats = indexedCols // check for included cols val writerSchemaOpt = LogFileColStatsTestUtil.getSchemaForTable(metaClient) - val latestCompletedCommit = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + val latestCompletedCommit = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants().lastInstant().get().requestedTime baseFilesDf.union(getColStatsFromLogFiles(allLogFiles, latestCompletedCommit, scala.collection.JavaConverters.seqAsJavaList(colsToGenerateStats), metaClient, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/PartitionStatsIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/PartitionStatsIndexTestBase.scala index fdbfe7ef4dd8..caf203eed800 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/PartitionStatsIndexTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/PartitionStatsIndexTestBase.scala @@ -28,9 +28,10 @@ import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model.{ActionType, HoodieCommitMetadata, WriteOperationType} import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieInstant, MetadataConversionUtils} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig -import org.apache.hudi.functional.PartitionStatsIndexTestBase.{checkIfOverlapped} +import org.apache.hudi.functional.PartitionStatsIndexTestBase.checkIfOverlapped import org.apache.hudi.metadata.HoodieBackedTableMetadata import org.apache.hudi.storage.StoragePath import org.apache.hudi.testutils.HoodieSparkClientTestBase @@ -95,7 +96,7 @@ class PartitionStatsIndexTestBase extends HoodieSparkClientTestBase { protected def getLatestMetaClient(enforce: Boolean): HoodieTableMetaClient = { val lastInstant = String.format("%03d", new Integer(instantTime.incrementAndGet())) - if (enforce || metaClient.getActiveTimeline.lastInstant().get().getTimestamp.compareTo(lastInstant) < 0) { + if (enforce || metaClient.getActiveTimeline.lastInstant().get().requestedTime.compareTo(lastInstant) < 0) { println("Reloaded timeline") metaClient.reloadActiveTimeline() metaClient @@ -114,7 +115,7 @@ class PartitionStatsIndexTestBase extends HoodieSparkClientTestBase { } val writeConfig = getWriteConfig(hudiOpts) new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), writeConfig) - .rollback(lastInstant.getTimestamp) + .rollback(lastInstant.requestedTime) if (lastInstant.getAction != ActionType.clean.name()) { assertEquals(ActionType.rollback.name(), getLatestMetaClient(true).getActiveTimeline.lastInstant().get().getAction) @@ -133,15 +134,15 @@ class PartitionStatsIndexTestBase extends HoodieSparkClientTestBase { val lastInstant = getHoodieTable(metaClient, writeConfig).getCompletedCommitsTimeline.lastInstant().get() val metadataTableMetaClient = getHoodieTable(metaClient, writeConfig).getMetadataTable.asInstanceOf[HoodieBackedTableMetadata].getMetadataMetaClient val metadataTableLastInstant = metadataTableMetaClient.getCommitsTimeline.lastInstant().get() - assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, lastInstant.getFileName))) - assertTrue(storage.deleteFile(new StoragePath(metadataTableMetaClient.getMetaPath, metadataTableLastInstant.getFileName))) + assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant)))) + assertTrue(storage.deleteFile(new StoragePath(metadataTableMetaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(metadataTableLastInstant)))) mergedDfList = mergedDfList.take(mergedDfList.size - 1) } protected def deleteLastCompletedCommitFromTimeline(hudiOpts: Map[String, String]): Unit = { val writeConfig = getWriteConfig(hudiOpts) val lastInstant = getHoodieTable(metaClient, writeConfig).getCompletedCommitsTimeline.lastInstant().get() - assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, lastInstant.getFileName))) + assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant)))) mergedDfList = mergedDfList.take(mergedDfList.size - 1) } @@ -306,6 +307,6 @@ class PartitionStatsIndexTestBase extends HoodieSparkClientTestBase { object PartitionStatsIndexTestBase { // Check if two completed instants are overlapped in time. def checkIfOverlapped(a: HoodieInstant, b: HoodieInstant): Boolean = { - !(a.getCompletionTime.compareTo(b.getTimestamp) < 0 || a.getTimestamp.compareTo(b.getCompletionTime) > 0) + !(a.getCompletionTime.compareTo(b.requestedTime) < 0 || a.requestedTime.compareTo(b.getCompletionTime) > 0) } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala index ec5a52145bb9..0254ff590e0b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/RecordLevelIndexTestBase.scala @@ -25,6 +25,7 @@ import org.apache.hudi.common.config.{HoodieMetadataConfig, TypedProperties} import org.apache.hudi.common.model._ import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieInstantTimeGenerator, MetadataConversionUtils} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.metadata.{HoodieBackedTableMetadata, HoodieTableMetadataUtil, MetadataPartitionType} @@ -107,7 +108,7 @@ class RecordLevelIndexTestBase extends HoodieSparkClientTestBase { protected def getLatestMetaClient(enforce: Boolean): HoodieTableMetaClient = { val lastInsant = HoodieInstantTimeGenerator.getLastInstantTime - if (enforce || metaClient.getActiveTimeline.lastInstant().get().getTimestamp.compareTo(lastInsant) < 0) { + if (enforce || metaClient.getActiveTimeline.lastInstant().get().requestedTime.compareTo(lastInsant) < 0) { println("Reloaded timeline") metaClient.reloadActiveTimeline() metaClient @@ -126,7 +127,7 @@ class RecordLevelIndexTestBase extends HoodieSparkClientTestBase { } val writeConfig = getWriteConfig(hudiOpts) new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), writeConfig) - .rollback(lastInstant.getTimestamp) + .rollback(lastInstant.requestedTime) if (lastInstant.getAction != ActionType.clean.name()) { assertEquals(ActionType.rollback.name(), getLatestMetaClient(true).getActiveTimeline.lastInstant().get().getAction) @@ -145,16 +146,16 @@ class RecordLevelIndexTestBase extends HoodieSparkClientTestBase { val lastInstant = getHoodieTable(getLatestMetaClient(false), writeConfig).getCompletedCommitsTimeline.lastInstant().get() val metadataTableMetaClient = getHoodieTable(metaClient, writeConfig).getMetadataTable.asInstanceOf[HoodieBackedTableMetadata].getMetadataMetaClient val metadataTableLastInstant = metadataTableMetaClient.getCommitsTimeline.lastInstant().get() - assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, lastInstant.getFileName))) + assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant)))) assertTrue(storage.deleteFile(new StoragePath( - metadataTableMetaClient.getMetaPath, metadataTableLastInstant.getFileName))) + metadataTableMetaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(metadataTableLastInstant)))) mergedDfList = mergedDfList.take(mergedDfList.size - 1) } protected def deleteLastCompletedCommitFromTimeline(hudiOpts: Map[String, String]): Unit = { val writeConfig = getWriteConfig(hudiOpts) val lastInstant = getHoodieTable(getLatestMetaClient(false), writeConfig).getCompletedCommitsTimeline.lastInstant().get() - assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, lastInstant.getFileName))) + assertTrue(storage.deleteFile(new StoragePath(metaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant)))) mergedDfList = mergedDfList.take(mergedDfList.size - 1) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala index 1bfad5701209..cb0419cd8d14 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestBloomFiltersIndexSupport.scala @@ -154,7 +154,7 @@ class TestBloomFiltersIndexSupport extends HoodieSparkClientTestBase { private def getLatestDataFilesCount(opts: Map[String, String], includeLogFiles: Boolean = true) = { var totalLatestDataFiles = 0L - getTableFileSystemView(opts).getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + getTableFileSystemView(opts).getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( 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 a9a7a012f51b..14b4964a558e 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 @@ -32,6 +32,8 @@ import org.apache.hudi.common.model.{HoodieRecord, WriteOperationType} import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineUtils} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.testutils.RawTripTestPayload.{deleteRecordsToStrings, recordsToStrings} import org.apache.hudi.common.util.{ClusteringUtils, Option} @@ -785,7 +787,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup .save(basePath) val metaClient = createMetaClient(spark, basePath) - val instantTime = metaClient.getActiveTimeline.filterCompletedInstants().getInstantsAsStream.findFirst().get().getTimestamp + val instantTime = metaClient.getActiveTimeline.filterCompletedInstants().getInstantsAsStream.findFirst().get().requestedTime val record1FilePaths = storage.listDirectEntries(new StoragePath(basePath, dataGen.getPartitionPaths.head)) .asScala @@ -1866,7 +1868,9 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup // Last instant is clustering assertTrue(TimelineUtils.getCommitMetadata(lastInstant, metaClient.getActiveTimeline) .getOperationType.equals(WriteOperationType.CLUSTER)) - assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline, new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, lastInstant.getTimestamp))) + assertTrue(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline, + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, lastInstant.requestedTime), + INSTANT_GENERATOR)) lastClustering = lastInstant assertEquals( lastClustering, @@ -1874,7 +1878,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup } else { assertTrue(TimelineUtils.getCommitMetadata(lastInstant, metaClient.getActiveTimeline) .getOperationType.equals(WriteOperationType.INSERT_OVERWRITE)) - assertFalse(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline, lastInstant)) + assertFalse(ClusteringUtils.isClusteringInstant(metaClient.getActiveTimeline, lastInstant, metaClient.getTimelineLayout.getInstantGenerator)) assertEquals( lastClustering, metaClient.getActiveTimeline.getLastClusteringInstant.get) @@ -1888,7 +1892,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup if (firstClusteringState == HoodieInstant.State.INFLIGHT || firstClusteringState == HoodieInstant.State.REQUESTED) { // Move the clustering to inflight for testing - storage.deleteFile(new StoragePath(metaClient.getMetaPath, lastInstant.getFileName)) + storage.deleteFile(new StoragePath(metaClient.getMetaPath, INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant))) val inflightClustering = metaClient.reloadActiveTimeline.lastInstant.get assertTrue(inflightClustering.isInflight) assertEquals( @@ -1914,8 +1918,8 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup // This should not schedule any new clustering new SparkRDDWriteClient(context, writeConfig) .scheduleClustering(org.apache.hudi.common.util.Option.of(Map[String, String]().asJava)) - assertEquals(lastInstant.getTimestamp, - metaClient.reloadActiveTimeline.getCommitsTimeline.lastInstant.get.getTimestamp) + assertEquals(lastInstant.requestedTime, + metaClient.reloadActiveTimeline.getCommitsTimeline.lastInstant.get.requestedTime) } } val timeline = metaClient.reloadActiveTimeline diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala index 9323b9ce2ed9..ffd14a0fa31a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndex.scala @@ -32,6 +32,7 @@ import org.apache.hudi.config.{HoodieCleanConfig, HoodieCompactionConfig, Hoodie import org.apache.hudi.functional.ColumnStatIndexTestBase.ColumnStatsTestCase import org.apache.hudi.storage.StoragePath import org.apache.hudi.{ColumnStatsIndexSupport, DataSourceWriteOptions} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hudi.client.common.HoodieSparkEngineContext @@ -174,7 +175,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { } metaClient = HoodieTableMetaClient.reload(metaClient) - val latestCompletedCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + val latestCompletedCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().requestedTime // lets validate that we have log files generated in case of MOR table if (tableType == HoodieTableType.MERGE_ON_READ) { @@ -280,7 +281,7 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { } metaClient = HoodieTableMetaClient.reload(metaClient) - val latestCompletedCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + val latestCompletedCommit = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().requestedTime // updates a subset which are not deleted and enable col stats and validate bootstrap doWriteAndValidateColumnStats(ColumnStatsTestParams(testCase, metadataOpts1, commonOpts, @@ -318,25 +319,25 @@ class TestColumnStatsIndex extends ColumnStatIndexTestBase { } else { metaClient.getStorage.listFiles(new StoragePath(metaClient.getBasePath, "9")) } - val baseFileFileStatus = dataFiles.stream().filter(fileStatus => fileStatus.getPath.getName.contains(lastCompletedCommit.getTimestamp)).findFirst().get() + val baseFileFileStatus = dataFiles.stream().filter(fileStatus => fileStatus.getPath.getName.contains(lastCompletedCommit.requestedTime)).findFirst().get() baseFileName = baseFileFileStatus.getPath.getName } - val latestCompletedFileName = lastCompletedCommit.getFileName + val latestCompletedFileName = INSTANT_FILE_NAME_GENERATOR.getFileName(lastCompletedCommit) metaClient.getStorage.deleteFile(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/" + latestCompletedFileName)) // re-create marker for the deleted file. if (tableType == HoodieTableType.MERGE_ON_READ) { if (StringUtils.isNullOrEmpty(partitionCol)) { - metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.getTimestamp + "/" + logFileName + ".marker.APPEND")) + metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.requestedTime + "/" + logFileName + ".marker.APPEND")) } else { - metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.getTimestamp + "/9/" + logFileName + ".marker.APPEND")) + metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.requestedTime + "/9/" + logFileName + ".marker.APPEND")) } } else { if (StringUtils.isNullOrEmpty(partitionCol)) { - metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.getTimestamp + "/" + baseFileName + ".marker.MERGE")) + metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.requestedTime + "/" + baseFileName + ".marker.MERGE")) } else { - metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.getTimestamp + "/9/" + baseFileName + ".marker.MERGE")) + metaClient.getStorage.create(new StoragePath(metaClient.getBasePath.toString + "/.hoodie/.temp/" + lastCompletedCommit.requestedTime + "/9/" + baseFileName + ".marker.MERGE")) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala index effda5af0164..78e04b2f417f 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestColumnStatsIndexWithSQL.scala @@ -237,7 +237,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase { var commonOpts = opts val inputDF1 = spark.read.format("hudi") .options(commonOpts) - .option("as.of.instant", metaClient.getActiveTimeline.getInstants.get(1).getTimestamp) + .option("as.of.instant", metaClient.getActiveTimeline.getInstants.get(1).requestedTime) .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL) .option(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key, "false") .load(basePath) @@ -286,7 +286,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase { fsView.loadAllPartitions() fsView.getPartitionPaths.asScala.flatMap { partitionPath => val relativePath = FSUtils.getRelativePartitionPath(metaClient.getBasePath, partitionPath) - fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, metaClient.reloadActiveTimeline().lastInstant().get().getTimestamp).iterator().asScala.toSeq + fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, metaClient.reloadActiveTimeline().lastInstant().get().requestedTime).iterator().asScala.toSeq }.foreach( slice => totalLatestDataFiles += (if (includeLogFiles) slice.getLogFiles.count() else 0) + (if (slice.getBaseFile.isPresent) 1 else 0)) @@ -356,7 +356,7 @@ class TestColumnStatsIndexWithSQL extends ColumnStatIndexTestBase { private def createSQLTable(hudiOpts: Map[String, String], queryType: String): Unit = { val opts = hudiOpts + ( DataSourceReadOptions.QUERY_TYPE.key -> queryType, - DataSourceReadOptions.START_COMMIT.key() -> metaClient.getActiveTimeline.getInstants.get(0).getTimestamp.replaceFirst(".", "0") + DataSourceReadOptions.START_COMMIT.key() -> metaClient.getActiveTimeline.getInstants.get(0).requestedTime.replaceFirst(".", "0") ) val inputDF1 = spark.read.format("hudi").options(opts).load(basePath) inputDF1.createOrReplaceTempView("tbl") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala index 4bf643bb2398..754535ae0bca 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestHoodieActiveTimeline.scala @@ -20,6 +20,7 @@ package org.apache.hudi.functional import org.apache.hudi.common.model.HoodieFileFormat import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.testutils.HoodieTestUtils import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieIOException @@ -34,7 +35,7 @@ import java.io.FileNotFoundException import scala.collection.JavaConverters._ /** - * Tests on HoodieActionTimeLine using the real hudi table. + * Tests on HoodieTimeline using the real hudi table. */ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { @@ -93,7 +94,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret1 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret1.isPresent) val (instant1, commitMetadata1) = (ret1.get().getLeft, ret1.get().getRight) - assertEquals(instant1.getTimestamp, commit1Time) + assertEquals(instant1.requestedTime, commit1Time) val relativePath1 = commitMetadata1.getFileIdAndRelativePaths.values().stream().findAny().get() assert(relativePath1.contains(commit1Time)) assert(relativePath1.contains(HoodieFileFormat.PARQUET.getFileExtension)) @@ -116,7 +117,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret2 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret2.isPresent) val (instant2, commitMetadata2) = (ret2.get().getLeft, ret2.get().getRight) - assertEquals(instant2.getTimestamp, commit1Time) + assertEquals(instant2.requestedTime, commit1Time) val relativePath2 = commitMetadata2.getFileIdAndRelativePaths.values().stream().findAny().get() assert(relativePath2.contains(commit1Time)) assert(relativePath2.contains(HoodieFileFormat.PARQUET.getFileExtension)) @@ -136,7 +137,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret3 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret3.isPresent) val (instant3, commitMetadata3) = (ret3.get().getLeft, ret3.get().getRight) - assertEquals(instant3.getTimestamp, commit3Time) + assertEquals(instant3.requestedTime, commit3Time) val relativePath3 = commitMetadata3.getFileIdAndRelativePaths.values().stream().findAny().get() assert(relativePath3.contains(commit3Time)) assert(relativePath3.contains(HoodieFileFormat.PARQUET.getFileExtension)) @@ -163,7 +164,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret1 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret1.isPresent) val (instant1, commitMetadata1) = (ret1.get().getLeft, ret1.get().getRight) - assertEquals(instant1.getTimestamp, commit1Time) + assertEquals(instant1.requestedTime, commit1Time) val relativePath1 = commitMetadata1.getFileIdAndRelativePaths.values().stream().findAny().get() assert(relativePath1.contains(commit1Time)) assert(relativePath1.contains(HoodieFileFormat.PARQUET.getFileExtension)) @@ -183,7 +184,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret2 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret2.isPresent) val (instant2, commitMetadata2) = (ret2.get().getLeft, ret2.get().getRight) - assertEquals(instant2.getTimestamp, commit2Time) + assertEquals(instant2.requestedTime, commit2Time) val relativePath2 = commitMetadata2.getFileIdAndRelativePaths.values().stream().findAny().get() // deltacommit: .log file should contain the timestamp of it's instant time. assert(relativePath2.contains(commit2Time)) @@ -205,7 +206,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret3 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret3.isPresent) val (instant3, commitMetadata3) = (ret3.get().getLeft, ret3.get().getRight) - assertEquals(instant3.getTimestamp, commit3Time) + assertEquals(instant3.requestedTime, commit3Time) val relativePath3 = commitMetadata3.getFileIdAndRelativePaths.values().stream().findAny().get() assert(relativePath3.contains(commit3Time)) assert(relativePath3.contains(HoodieFileFormat.PARQUET.getFileExtension)) @@ -224,7 +225,7 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val ret4 = activeTimeline.getLastCommitMetadataWithValidData() assert(ret4.isPresent) val (instant4, commitMetadata4) = (ret4.get().getLeft, ret4.get().getRight) - assertEquals(instant4.getTimestamp, commit4Time) + assertEquals(instant4.requestedTime, commit4Time) val relativePath4 = commitMetadata4.getFileIdAndRelativePaths.values().stream().findAny().get() // deltacommit: .log file should contain the timestamp of it's instant time. assert(relativePath4.contains(commit4Time)) @@ -247,7 +248,8 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { val activeTimeline = metaClient.getActiveTimeline assertNotNull(activeTimeline.getInstantDetails(activeTimeline.lastInstant().get())) try { - activeTimeline.getInstantDetails(new HoodieInstant(true, HoodieTimeline.CLUSTERING_ACTION, metaClient.createNewInstantTime())) + activeTimeline.getInstantDetails(HoodieTestUtils.INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, + HoodieTimeline.CLUSTERING_ACTION, metaClient.createNewInstantTime())) } catch { // org.apache.hudi.common.util.ClusteringUtils.getRequestedReplaceMetadata depends upon this behaviour // where FileNotFoundException is the cause of exception thrown by the API getInstantDetails diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala index f3fec521f066..703138f0c7c1 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestIncrementalReadByStateTransitionTime.scala @@ -84,7 +84,7 @@ class TestIncrementalReadByStateTransitionTime extends HoodieSparkClientTestBase .option(DataSourceReadOptions.QUERY_TYPE.key(), DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.START_COMMIT.key(), "000") .option(DataSourceReadOptions.INCREMENTAL_READ_HANDLE_HOLLOW_COMMIT.key(), USE_TRANSITION_TIME.name()) - .option(DataSourceReadOptions.END_COMMIT.key(), firstInstant.getTimestamp) + .option(DataSourceReadOptions.END_COMMIT.key(), firstInstant.requestedTime) .load(basePath) .count() 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 54774a9a3ac5..bd05f542a14a 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 @@ -20,14 +20,13 @@ package org.apache.hudi.functional import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions} import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.model.HoodieTableType -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, InstantComparison} import org.apache.hudi.common.table.timeline.HoodieInstantTimeGenerator.instantTimeMinusMillis -import org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN +import InstantComparison.compareTimestamps import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings import org.apache.hudi.config.HoodieWriteConfig import org.apache.hudi.exception.HoodieIOException import org.apache.hudi.testutils.HoodieSparkClientTestBase - import org.apache.spark.SparkException import org.apache.spark.sql.{SaveMode, SparkSession} import org.junit.jupiter.api.{AfterEach, BeforeEach} @@ -118,8 +117,8 @@ class TestIncrementalReadWithFullTableScan extends HoodieSparkClientTestBase { val startOutOfRangeCommitTs = hoodieMetaClient.createNewInstantTime() val endOutOfRangeCommitTs = hoodieMetaClient.createNewInstantTime() - assertTrue(HoodieTimeline.compareTimestamps(startOutOfRangeCommitTs, GREATER_THAN, completedCommits.lastInstant().get().getTimestamp)) - assertTrue(HoodieTimeline.compareTimestamps(endOutOfRangeCommitTs, GREATER_THAN, completedCommits.lastInstant().get().getTimestamp)) + assertTrue(compareTimestamps(startOutOfRangeCommitTs, InstantComparison.GREATER_THAN, completedCommits.lastInstant().get().requestedTime)) + assertTrue(compareTimestamps(endOutOfRangeCommitTs, InstantComparison.GREATER_THAN, completedCommits.lastInstant().get().requestedTime)) // Test both start and end commits are archived runIncrementalQueryAndCompare(startArchivedCompletionTs, endArchivedCompletionTs, 1, true) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index 622d2c8e7048..9e3c09edb764 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -1184,7 +1184,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin .mode(SaveMode.Overwrite) .save(basePath) metaClient = createMetaClient(spark, basePath) - val commit1Time = metaClient.getActiveTimeline.lastInstant().get().getTimestamp + val commit1Time = metaClient.getActiveTimeline.lastInstant().get().requestedTime val dataGen2 = new HoodieTestDataGenerator(Array("2022-01-02")) val records2 = recordsToStrings(dataGen2.generateInserts("002", 60)).asScala.toSeq @@ -1193,7 +1193,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin .options(options) .mode(SaveMode.Append) .save(basePath) - val commit2Time = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commit2Time = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val records3 = recordsToStrings(dataGen2.generateUniqueUpdates("003", 20)).asScala.toSeq val inputDF3 = spark.read.json(spark.sparkContext.parallelize(records3, 2)) @@ -1201,7 +1201,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin .options(options) .mode(SaveMode.Append) .save(basePath) - val commit3Time = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commit3Time = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val commit3CompletionTime = metaClient.reloadActiveTimeline.lastInstant().get().getCompletionTime val pathForROQuery = getPathForROQuery(basePath, !enableFileIndex, 3) @@ -1516,7 +1516,7 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin // do a time travel query with data skipping enabled val timeTravelDF = spark.read.format("hudi") .options(readOpts) - .option("as.of.instant", firstInstant.getTimestamp) + .option("as.of.instant", firstInstant.requestedTime) .load(basePath) // there should still be 3 records in time travel view assertEquals(3, timeTravelDF.count()) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala index 74686933921d..cd4a6601417a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataRecordIndex.scala @@ -104,7 +104,7 @@ class TestMetadataRecordIndex extends HoodieSparkClientTestBase { operation = UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(getLatestClusteringInstant().get().getTimestamp.compareTo(lastClusteringInstant.get().getTimestamp) > 0) + assertTrue(getLatestClusteringInstant().get().requestedTime.compareTo(lastClusteringInstant.get().requestedTime) > 0) validateDataAndRecordIndices(hudiOpts) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala index 736926cd6979..28ada0c31b16 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMetadataTableWithSparkDataSource.scala @@ -261,7 +261,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn val timelineT0 = metaClient.getActiveTimeline assertEquals(3, timelineT0.countInstants()) assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, timelineT0.lastInstant().get().getAction) - val t0 = timelineT0.lastInstant().get().getTimestamp + val t0 = timelineT0.lastInstant().get().requestedTime val filesT0 = getFiles(basePath) assertEquals(3, filesT0.size) @@ -287,7 +287,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn val timelineT1 = metaClient.reloadActiveTimeline() assertEquals(4, timelineT1.countInstants()) assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, timelineT1.lastInstant().get().getAction) - val t1 = timelineT1.lastInstant().get().getTimestamp + val t1 = timelineT1.lastInstant().get().requestedTime val filesT1 = getFiles(basePath) assertEquals(6, filesT1.size) @@ -314,7 +314,7 @@ class TestMetadataTableWithSparkDataSource extends SparkClientFunctionalTestHarn val timelineT2 = metaClient.reloadActiveTimeline() assertEquals(5, timelineT2.countInstants()) assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, timelineT2.lastInstant().get().getAction) - val t2 = timelineT2.lastInstant().get().getTimestamp + val t2 = timelineT2.lastInstant().get().requestedTime val filesT2 = getFiles(basePath) assertEquals(9, filesT2.size) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala index 07d65a19a63c..3147736e7edd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestParquetColumnProjection.scala @@ -275,8 +275,8 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with */ val hoodieMetaClient = createMetaClient(spark, tablePath) val completedCommits = hoodieMetaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() - val startUnarchivedCommitTs = completedCommits.nthInstant(1).get().getTimestamp //deltacommit2 - val endUnarchivedCommitTs = completedCommits.nthInstant(5).get().getTimestamp //deltacommit6 + val startUnarchivedCommitTs = completedCommits.nthInstant(1).get().requestedTime //deltacommit2 + val endUnarchivedCommitTs = completedCommits.nthInstant(5).get().requestedTime //deltacommit6 val readOpts = defaultWriteOpts ++ Map( "path" -> tablePath, @@ -301,8 +301,8 @@ class TestParquetColumnProjection extends SparkClientFunctionalTestHarness with val hoodieMetaClient = createMetaClient(spark, tablePath) val completedCommits = hoodieMetaClient.getCommitsAndCompactionTimeline.filterCompletedInstants() - val startUnarchivedCommitTs = (completedCommits.nthInstant(1).get().getTimestamp.toLong - 1L).toString - val endUnarchivedCommitTs = completedCommits.nthInstant(3).get().getTimestamp //commit + val startUnarchivedCommitTs = (completedCommits.nthInstant(1).get().requestedTime.toLong - 1L).toString + val endUnarchivedCommitTs = completedCommits.nthInstant(3).get().requestedTime //commit val readOpts = defaultWriteOpts ++ Map( "path" -> tablePath, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala index fa1f4c3baa89..9e886f0b7540 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndex.scala @@ -321,7 +321,7 @@ class TestPartitionStatsIndex extends PartitionStatsIndexTestBase { doWriteAndValidateDataAndPartitionStats(hudiOpts, operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(getLatestClusteringInstant.get().getTimestamp.compareTo(lastClusteringInstant.get().getTimestamp) > 0) + assertTrue(getLatestClusteringInstant.get().requestedTime.compareTo(lastClusteringInstant.get().requestedTime) > 0) assertEquals(getLatestClusteringInstant, metaClient.getActiveTimeline.lastInstant()) // We are validating rollback of a DT clustering instant here rollbackLastInstant(hudiOpts) @@ -358,9 +358,9 @@ class TestPartitionStatsIndex extends PartitionStatsIndexTestBase { assertTrue(metaClient.getTableConfig.getMetadataPartitions.contains(MetadataPartitionType.RECORD_INDEX.getPartitionPath)) // Do a savepoint val writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), getWriteConfig(hudiOpts)) - writeClient.savepoint(firstCompletedInstant.get().getTimestamp, "testUser", "savepoint to first commit") - val savepointTimestamp = metaClient.reloadActiveTimeline().getSavePointTimeline.filterCompletedInstants().lastInstant().get().getTimestamp - assertEquals(firstCompletedInstant.get().getTimestamp, savepointTimestamp) + writeClient.savepoint(firstCompletedInstant.get().requestedTime, "testUser", "savepoint to first commit") + val savepointTimestamp = metaClient.reloadActiveTimeline().getSavePointTimeline.filterCompletedInstants().lastInstant().get().requestedTime + assertEquals(firstCompletedInstant.get().requestedTime, savepointTimestamp) // Restore to savepoint writeClient.restoreToSavepoint(savepointTimestamp) // verify restore completed @@ -405,11 +405,11 @@ class TestPartitionStatsIndex extends PartitionStatsIndexTestBase { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - HoodieCommitMetadata.fromBytes(compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles(MetadataPartitionType.PARTITION_STATS.getPartitionPath) - .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().getTimestamp))) + .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().requestedTime))) .findAny() assertTrue(compactionBaseFile.isPresent) } finally { @@ -450,7 +450,7 @@ class TestPartitionStatsIndex extends PartitionStatsIndexTestBase { private def getLatestDataFilesCount(opts: Map[String, String], includeLogFiles: Boolean = true) = { var totalLatestDataFiles = 0L - getTableFileSystemView(opts).getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + getTableFileSystemView(opts).getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala index f585b720fcd2..dbd764ce9763 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestPartitionStatsIndexWithSql.scala @@ -572,7 +572,7 @@ class TestPartitionStatsIndexWithSql extends HoodieSparkSqlTestBase { var totalLatestDataFiles = 0L val fsView: HoodieMetadataFileSystemView = getTableFileSystemView(metaClient) try { - fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala index d89fa0133479..f0333c515a12 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndex.scala @@ -312,8 +312,8 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { doWriteAndValidateDataAndRecordIndex(hudiOpts, operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(getLatestMetaClient(false).getActiveTimeline.getCleanerTimeline.lastInstant().get().getTimestamp - .compareTo(lastCleanInstant.get().getTimestamp) > 0) + assertTrue(getLatestMetaClient(false).getActiveTimeline.getCleanerTimeline.lastInstant().get().requestedTime + .compareTo(lastCleanInstant.get().requestedTime) > 0) var rollbackedInstant: Option[HoodieInstant] = Option.empty while (rollbackedInstant.isEmpty || rollbackedInstant.get.getAction != ActionType.clean.name()) { @@ -348,11 +348,11 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { doWriteAndValidateDataAndRecordIndex(hudiOpts, operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(getLatestCompactionInstant().get().getTimestamp.compareTo(lastCompactionInstant.get().getTimestamp) > 0) + assertTrue(getLatestCompactionInstant().get().requestedTime.compareTo(lastCompactionInstant.get().requestedTime) > 0) lastCompactionInstant = getLatestCompactionInstant() var rollbackedInstant: Option[HoodieInstant] = Option.empty - while (rollbackedInstant.isEmpty || rollbackedInstant.get.getTimestamp != lastCompactionInstant.get().getTimestamp) { + while (rollbackedInstant.isEmpty || rollbackedInstant.get.requestedTime != lastCompactionInstant.get().requestedTime) { // rollback compaction instant rollbackedInstant = Option.apply(rollbackLastInstant(hudiOpts)) } @@ -385,7 +385,7 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { operation = DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL, saveMode = SaveMode.Append) - assertTrue(getLatestClusteringInstant().get().getTimestamp.compareTo(lastClusteringInstant.get().getTimestamp) > 0) + assertTrue(getLatestClusteringInstant().get().requestedTime.compareTo(lastClusteringInstant.get().requestedTime) > 0) assertEquals(getLatestClusteringInstant(), metaClient.getActiveTimeline.lastInstant()) // We are validating rollback of a DT clustering instant here rollbackLastInstant(hudiOpts) @@ -482,11 +482,11 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - HoodieCommitMetadata.fromBytes(compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles(MetadataPartitionType.RECORD_INDEX.getPartitionPath) - .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().getTimestamp))) + .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().requestedTime))) .findAny() assertTrue(compactionBaseFile.isPresent) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala index 3507c10d0468..1e36d4507709 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestRecordLevelIndexWithSQL.scala @@ -152,7 +152,7 @@ class TestRecordLevelIndexWithSQL extends RecordLevelIndexTestBase { private def getLatestDataFilesCount(opts: Map[String, String], includeLogFiles: Boolean = true) = { var totalLatestDataFiles = 0L Using(getTableFileSystenView(opts)) { fsView => - fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala index ee11d35f7d4e..227ee4a5acdf 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSecondaryIndexPruning.scala @@ -660,11 +660,11 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - HoodieCommitMetadata.fromBytes(compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles("secondary_index_idx_not_record_key_col") - .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().getTimestamp))) + .filter(JavaConversions.getPredicate((f: HoodieBaseFile) => f.getCommitTime.equals(lastCompactionInstant.get().requestedTime))) .findAny() assertTrue(compactionBaseFile.isPresent) } finally { @@ -1154,9 +1154,9 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness { // Do a savepoint val firstCompletedInstant = metaClient.getActiveTimeline.getCommitsTimeline.filterCompletedInstants().lastInstant() val writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(jsc), getWriteConfig(hudiOpts)) - writeClient.savepoint(firstCompletedInstant.get().getTimestamp, "testUser", "savepoint to first commit") - val savepointTimestamp = metaClient.reloadActiveTimeline().getSavePointTimeline.filterCompletedInstants().lastInstant().get().getTimestamp - assertEquals(firstCompletedInstant.get().getTimestamp, savepointTimestamp) + writeClient.savepoint(firstCompletedInstant.get().requestedTime, "testUser", "savepoint to first commit") + val savepointTimestamp = metaClient.reloadActiveTimeline().getSavePointTimeline.filterCompletedInstants().lastInstant().get().requestedTime + assertEquals(firstCompletedInstant.get().requestedTime, savepointTimestamp) // Restore to savepoint writeClient.restoreToSavepoint(savepointTimestamp) // verify restore completed @@ -1213,7 +1213,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness { var totalLatestDataFiles = 0L val fsView: HoodieMetadataFileSystemView = getTableFileSystemView(opts) try { - fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala index a7237e70795f..d220c6f92061 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSevenToEightUpgrade.scala @@ -34,10 +34,12 @@ class TestSevenToEightUpgrade extends RecordLevelIndexTestBase { @EnumSource(classOf[HoodieTableType]) def testPartitionFieldsWithUpgrade(tableType: HoodieTableType): Unit = { val partitionFields = "partition:simple" + // Downgrade handling for metadata not yet ready. val hudiOpts = commonOpts ++ Map( DataSourceWriteOptions.TABLE_TYPE.key -> tableType.name(), DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> KeyGeneratorType.CUSTOM.getClassName, - DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> partitionFields) + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> partitionFields, + "hoodie.metadata.enable" -> "false") doWriteAndValidateDataAndRecordIndex(hudiOpts, operation = DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL, diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSixToFiveDowngradeHandler.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSixToFiveDowngradeHandler.scala index 16ac7cd4dfd7..48db4ddd1b92 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSixToFiveDowngradeHandler.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSixToFiveDowngradeHandler.scala @@ -117,7 +117,7 @@ class TestSixToFiveDowngradeHandler extends RecordLevelIndexTestBase { getAllPartititonPaths(fsView).asScala.flatMap { partitionPath => val relativePath = FSUtils.getRelativePartitionPath(metaClient.getBasePath, partitionPath) fsView.getLatestMergedFileSlicesBeforeOrOn(relativePath, getLatestMetaClient(false) - .getActiveTimeline.lastInstant().get().getTimestamp).iterator().asScala.toSeq + .getActiveTimeline.lastInstant().get().requestedTime).iterator().asScala.toSeq }.foreach( slice => if (slice.getLogFiles.count() > 0) { numFileSlicesWithLogFiles += 1 diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala index 402bcbbfd124..290de6509726 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestStructuredStreaming.scala @@ -470,7 +470,7 @@ class TestStructuredStreaming extends HoodieSparkClientTestBase { .getTimelineOfActions(CollectionUtils.createSet(instantAction)) .filterCompletedInstants .lastInstant - .get.getTimestamp + .get.requestedTime } private def streamingWrite(schema: StructType, sourcePath: String, destPath: String, hudiOptions: Map[String, String], checkpoint: String): Unit = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala index 504d7a53aacb..e88bc78cc371 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestTimeTravelQuery.scala @@ -21,14 +21,13 @@ import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_READ} import org.apache.hudi.common.model.{HoodieCleaningPolicy, HoodieTableType} import org.apache.hudi.common.table.TableSchemaResolver -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.common.table.timeline.{HoodieActiveTimeline, TimelineUtils} import org.apache.hudi.common.testutils.HoodieTestTable import org.apache.hudi.config.{HoodieArchivalConfig, HoodieCleanConfig, HoodieCompactionConfig, HoodieWriteConfig} import org.apache.hudi.exception.ExceptionUtil.getRootCause import org.apache.hudi.exception.HoodieTimeTravelException import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, ScalaAssertionSupport} - import org.apache.spark.sql.SaveMode.{Append, Overwrite} import org.apache.spark.sql.{DataFrame, Row, SaveMode, SparkSession} import org.junit.jupiter.api.Assertions.{assertEquals, assertNotNull, assertNull, assertTrue} @@ -230,17 +229,17 @@ class TestTimeTravelQuery extends HoodieSparkClientTestBase with ScalaAssertionS private def writeBatch(df: DataFrame, options: Map[String, String], mode: SaveMode = Append): String = { df.write.format("hudi").options(options).mode(mode).save(basePath) metaClient.reloadActiveTimeline() - metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().getTimestamp + metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get().requestedTime } private def defaultDateTimeFormat(queryInstant: String): String = { - val date = HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) + val date = TimelineUtils.parseDateFromInstantTime(queryInstant) val format = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss.SSS") format.format(date) } private def defaultDateFormat(queryInstant: String): String = { - val date = HoodieActiveTimeline.parseDateFromInstantTime(queryInstant) + val date = TimelineUtils.parseDateFromInstantTime(queryInstant) val format = new SimpleDateFormat("yyyy-MM-dd") format.format(date) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala index 7f7451232cf8..3ce7982e8800 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/HoodieCDCTestBase.scala @@ -94,7 +94,7 @@ abstract class HoodieCDCTestBase extends HoodieSparkClientTestBase { * whether this instant will create a cdc log file. */ protected def hasCDCLogFile(instant: HoodieInstant): Boolean = { - val commitMetadata = HoodieCommitMetadata.fromBytes( + val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), classOf[HoodieCommitMetadata] ) @@ -110,7 +110,7 @@ abstract class HoodieCDCTestBase extends HoodieSparkClientTestBase { * extract a list of cdc log file. */ protected def getCDCLogFile(instant: HoodieInstant): List[String] = { - val commitMetadata = HoodieCommitMetadata.fromBytes( + val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), classOf[HoodieCommitMetadata] ) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala index 2864e902a930..e62b48a1c2f6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCDataFrameSuite.scala @@ -81,7 +81,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() // all the data is new-coming, it will write out cdc log files. assertFalse(hasCDCLogFile(instant1)) - val commitTime1 = instant1.getTimestamp + val commitTime1 = instant1.requestedTime val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) @@ -105,7 +105,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { checkCDCDataForInsertOrUpdate(loggingMode, cdcSchema, dataSchema, cdcDataFromCDCLogFile2, hoodieRecords2, HoodieCDCOperation.UPDATE) - val commitTime2 = instant2.getTimestamp + val commitTime2 = instant2.requestedTime var currentSnapshotData = spark.read.format("hudi").load(basePath) // at the last commit, 100 records are inserted. val insertedCnt2 = currentSnapshotData.count() - 100 @@ -128,7 +128,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant3 = metaClient.reloadActiveTimeline.lastInstant().get() // only part of data are deleted and some data will write back to the file. // it will write out cdc log files. But instant3 is the clustering instant, not the delete one. so we omit to test. - val commitTime3 = instant3.getTimestamp + val commitTime3 = instant3.requestedTime currentSnapshotData = spark.read.format("hudi").load(basePath) // here we use `commitTime2` to query the change data in commit 3. // because `commitTime3` is the ts of the clustering operation, not the delete operation. @@ -157,7 +157,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // and all the new data will write out some new file groups. // it will NOT write out cdc log files assertFalse(hasCDCLogFile(instant4)) - val commitTime4 = instant4.getTimestamp + val commitTime4 = instant4.requestedTime val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) val insertedCnt4 = 50 val deletedCnt4 = currentSnapshotData.count() @@ -194,7 +194,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .save(basePath) val instant7 = metaClient.reloadActiveTimeline.getCommitsTimeline.lastInstant().get() // part of data are updated, it will write out cdc log files. - val cdcDataOnly7 = cdcDataFrame((instant7.getTimestamp.toLong - 1).toString) + val cdcDataOnly7 = cdcDataFrame((instant7.requestedTime.toLong - 1).toString) val currentData = spark.read.format("hudi").load(basePath) val insertedCnt7 = currentData.count() - 60 val updatedCnt7 = 30 - insertedCnt7 @@ -221,7 +221,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // and all the new data will write out some new file groups. // it will NOT write out cdc log files assertFalse(hasCDCLogFile(instant8)) - val commitTime8 = instant8.getTimestamp + val commitTime8 = instant8.requestedTime val cdcDataOnly8 = cdcDataFrame((commitTime8.toLong - 1).toString) assertCDCOpCnt(cdcDataOnly8, 20, 0, 0) totalInsertedCnt += 20 @@ -271,7 +271,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() // all the data is new-coming, it will write out cdc log files. assertFalse(hasCDCLogFile(instant1)) - val commitTime1 = instant1.getTimestamp + val commitTime1 = instant1.requestedTime val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) @@ -294,7 +294,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { assertEquals(cdcDataFromCDCLogFile2.count(r => r.getData.asInstanceOf[GenericRecord].get(0).toString == "u"), 30) assertEquals(cdcDataFromCDCLogFile2.count(r => r.getData.asInstanceOf[GenericRecord].get(0).toString == "i"), 20) - val commitTime2 = instant2.getTimestamp + val commitTime2 = instant2.requestedTime var currentSnapshotData = spark.read.format("hudi").load(basePath) // at the last commit, 100 records are inserted. val insertedCnt2 = currentSnapshotData.count() - 100 @@ -317,7 +317,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant3 = metaClient.reloadActiveTimeline.lastInstant().get() // in cases that there is log files, it will NOT write out cdc log files. // But instant3 is the compaction instant, not the delete one. so we omit to test. - val commitTime3 = instant3.getTimestamp + val commitTime3 = instant3.requestedTime currentSnapshotData = spark.read.format("hudi").load(basePath) // here we use `commitTime2` to query the change data in commit 3. // because `commitTime3` is the ts of the clustering operation, not the delete operation. @@ -342,7 +342,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // all the new data will write out some new file groups. // it will NOT write out cdc log files assertFalse(hasCDCLogFile(instant4)) - val commitTime4 = instant4.getTimestamp + val commitTime4 = instant4.requestedTime val cntForInstant4 = spark.read.format("hudi").load(basePath).count() val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) val insertedCnt4 = 100 @@ -365,7 +365,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant5 = metaClient.reloadActiveTimeline.lastInstant().get() // in cases that there is log files, it will NOT write out cdc log files. // But instant9 is the clustering instant, not the upsert one. so we omit to test. - val commitTime5 = instant5.getTimestamp + val commitTime5 = instant5.requestedTime // here we use `commitTime4` to query the change data in commit 5. // because `commitTime5` is the ts of the clean operation, not the upsert operation. val cdcDataOnly5 = cdcDataFrame(commitTime4) @@ -397,7 +397,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // and all the new data will write out some new file groups. // it will NOT write out cdc log files assertFalse(hasCDCLogFile(instant6)) - val commitTime6 = instant6.getTimestamp + val commitTime6 = instant6.requestedTime val cntForInstant6 = spark.read.format("hudi").load(basePath).count() val cdcDataOnly6 = cdcDataFrame((commitTime6.toLong - 1).toString) assertCDCOpCnt(cdcDataOnly6, 70, 0, cntForInstant5) @@ -424,7 +424,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .mode(SaveMode.Append) .save(basePath) val instant8 = metaClient.reloadActiveTimeline.lastInstant().get() - val commitTime8 = instant8.getTimestamp + val commitTime8 = instant8.requestedTime totalInsertedCnt += 3 // 8. Upsert Operation With Clean Operation @@ -442,7 +442,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant9 = metaClient.reloadActiveTimeline.lastInstant().get() // in cases that there is log files, it will NOT write out cdc log files. // But instant9 is the clean instant, not the upsert one. so we omit to test. - val commitTime9 = instant9.getTimestamp + val commitTime9 = instant9.requestedTime val cntForInstant9 = spark.read.format("hudi").load(basePath).count() val cdcDataOnly9 = cdcDataFrame(commitTime8) val insertedCnt9 = cntForInstant9 - cntForInstant6 - 10 @@ -496,7 +496,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { val instant1 = metaClient.reloadActiveTimeline.lastInstant().get() // all the data is new-coming, it will write out cdc log files. assertFalse(hasCDCLogFile(instant1)) - val commitTime1 = instant1.getTimestamp + val commitTime1 = instant1.requestedTime val cdcDataOnly1 = cdcDataFrame((commitTime1.toLong - 1).toString) assertCDCOpCnt(cdcDataOnly1, 100, 0, 0) @@ -513,7 +513,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // and all the new data will write out some new file groups. // it will NOT write out cdc log files assertFalse(hasCDCLogFile(instant2)) - val commitTime2 = instant2.getTimestamp + val commitTime2 = instant2.requestedTime val insertedCnt2 = 30 val deletedCnt2 = partitionToCnt(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH) val cdcDataOnly2 = cdcDataFrame((commitTime2.toLong - 1).toString) @@ -535,7 +535,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { // the files belongs to this partition will be replaced directly. // it will NOT write out cdc log files. assertFalse(hasCDCLogFile(instant3)) - val commitTime3 = instant3.getTimestamp + val commitTime3 = instant3.requestedTime val cntForInstant3 = spark.read.format("hudi").load(basePath).count() // here we use `commitTime2` to query the change data in commit 3. // because `commitTime3` is the ts of the clustering operation, not the delete operation. @@ -556,7 +556,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .mode(SaveMode.Append) .save(basePath) val instant4 = metaClient.reloadActiveTimeline.lastInstant().get() - val commitTime4 = instant4.getTimestamp + val commitTime4 = instant4.requestedTime val cntForInstant4 = spark.read.format("hudi").load(basePath).count() val cdcDataOnly4 = cdcDataFrame((commitTime4.toLong - 1).toString) val insertedCnt4 = cntForInstant4 - cntForInstant3 @@ -623,7 +623,7 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { checkCDCDataForInsertOrUpdate(loggingMode, cdcSchema, dataSchema, cdcDataFromCDCLogFile2, hoodieRecords2, HoodieCDCOperation.UPDATE) - val commitTime2 = instant2.getTimestamp + val commitTime2 = instant2.requestedTime var currentSnapshotData = spark.read.format("hudi").load(basePath) // at the last commit, 100 records are inserted. val insertedCnt2 = currentSnapshotData.count() - 100 @@ -803,8 +803,8 @@ class TestCDCDataFrameSuite extends HoodieCDCTestBase { .mode(SaveMode.Append).save(basePath) val metaClient = createMetaClient(spark, basePath) - val startTimeStamp = metaClient.reloadActiveTimeline().firstInstant().get.getTimestamp - val latestTimeStamp = metaClient.reloadActiveTimeline().lastInstant().get.getTimestamp + val startTimeStamp = metaClient.reloadActiveTimeline().firstInstant().get.requestedTime + val latestTimeStamp = metaClient.reloadActiveTimeline().lastInstant().get.requestedTime val result1 = spark.read.format("hudi") .option("hoodie.datasource.query.type", "incremental") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala index 947d62636633..3bbc6727ec04 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/cdc/TestCDCStreamingSuite.scala @@ -162,7 +162,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase { // check the change data about user_to_country_tbl for batch1 val detailOutput1 = spark.read.format("hudi").load(userToCountryTblPath) assert(detailOutput1.where("country = 'US'").count() == 5) - val ucTs1 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.getTimestamp + val ucTs1 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.requestedTime val ucDdcData1 = cdcDataFrame(userToCountryTblPath, (ucTs1.toLong - 1).toString, null) ucDdcData1.show(false) assertCDCOpCnt(ucDdcData1, 1, 2, 0) @@ -185,7 +185,7 @@ class TestCDCStreamingSuite extends HoodieCDCTestBase { stream2.processAllAvailable() // check the change data about user_to_country_tbl for batch2 - val ts2 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.getTimestamp + val ts2 = userToCountryMetaClient.reloadActiveTimeline().lastInstant.get.requestedTime val cdcData2 = cdcDataFrame(userToCountryTblPath, (ts2.toLong - 1).toString, null) cdcData2.show(false) assertCDCOpCnt(cdcData2, 2, 1, 0) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala index 16b14bf34799..5660dbec4b61 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala @@ -20,16 +20,18 @@ package org.apache.spark.sql.execution.benchmark import org.apache.hudi.DummyActiveAction import org.apache.hudi.client.common.HoodieJavaEngineContext -import org.apache.hudi.client.timeline.LSMTimelineWriter import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieCommitMetadata, HoodieTableType, WriteOperationType} import org.apache.hudi.common.table.timeline.{ActiveAction, CompletionTimeQueryView, HoodieArchivedTimeline, HoodieInstant, LSMTimeline} import org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata +import org.apache.hudi.common.testutils.HoodieTestUtils.{INSTANT_GENERATOR, TIMELINE_FACTORY} import org.apache.hudi.common.testutils.{HoodieTestTable, HoodieTestUtils} import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} import org.apache.hudi.index.HoodieIndex.IndexType import org.apache.hudi.table.HoodieJavaTable import org.apache.hadoop.fs.Path +import org.apache.hudi.client.timeline.versioning.v2.LSMTimelineWriter +import org.apache.hudi.common.table.timeline.versioning.v2.CompletionTimeQueryViewV2 import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} import java.util @@ -69,9 +71,9 @@ object LSMTimelineReadBenchmark extends HoodieBenchmarkBase { val instantTime = startTs + i + "" val completionTime = startTs + i + 1000 + "" val action = if (i % 2 == 0) "delta_commit" else "commit" - val instant = new HoodieInstant(HoodieInstant.State.COMPLETED, action, instantTime, completionTime) + val instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, action, instantTime, completionTime) val metadata: HoodieCommitMetadata = HoodieTestTable.of(metaClient).createCommitMetadata(instantTime, WriteOperationType.INSERT, util.Arrays.asList("par1", "par2"), 10, false) - val serializedMetadata = serializeCommitMetadata(metadata).get() + val serializedMetadata = serializeCommitMetadata(metaClient.getTimelineLayout.getCommitMetadataSerDe, metadata).get() instantBuffer.add(new DummyActiveAction(instant, serializedMetadata)) if (i % batchSize == 0) { // archive 10 instants each time @@ -83,10 +85,10 @@ object LSMTimelineReadBenchmark extends HoodieBenchmarkBase { val benchmark = new HoodieBenchmark("pref load archived instants", commitsNum, 3) benchmark.addCase("read slim instants") { _ => - new HoodieArchivedTimeline(metaClient) + TIMELINE_FACTORY.createArchivedTimeline(metaClient) } benchmark.addCase("read instants with commit metadata") { _ => - new HoodieArchivedTimeline(metaClient, startInstant) + TIMELINE_FACTORY.createArchivedTimeline(metaClient, startInstant) } // for scala compatibility val earliestStartTimeFunc: java.util.function.Function[String, String] = new java.util.function.Function[String, String] { @@ -95,7 +97,7 @@ object LSMTimelineReadBenchmark extends HoodieBenchmarkBase { } } benchmark.addCase("read start time") { _ => - new CompletionTimeQueryView(metaClient).getInstantTimes(startTs + 1 + 1000 + "", startTs + commitsNum + 1000 + "", earliestStartTimeFunc) + new CompletionTimeQueryViewV2(metaClient).getInstantTimes(startTs + 1 + 1000 + "", startTs + commitsNum + 1000 + "", earliestStartTimeFunc) } benchmark.run() val totalSize = LSMTimeline.latestSnapshotManifest(metaClient).getFiles.asScala diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestFunctionalIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestFunctionalIndex.scala index cf4b8ed8299f..859c150109e0 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestFunctionalIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestFunctionalIndex.scala @@ -790,7 +790,7 @@ class TestFunctionalIndex extends HoodieSparkSqlTestBase { // rollback the operation val lastCompletedInstant = metaClient.reloadActiveTimeline().getCommitsTimeline.filterCompletedInstants().lastInstant() val writeClient = new SparkRDDWriteClient(new HoodieSparkEngineContext(new JavaSparkContext(spark.sparkContext)), getWriteConfig(Map.empty, metaClient.getBasePath.toString)) - writeClient.rollback(lastCompletedInstant.get().getTimestamp) + writeClient.rollback(lastCompletedInstant.get().requestedTime) // validate the functional index checkAnswer(metadataSql)( // the last commit is rolledback so no records for that @@ -990,7 +990,7 @@ class TestFunctionalIndex extends HoodieSparkSqlTestBase { var totalLatestDataFiles = 0L val fsView: HoodieMetadataFileSystemView = getTableFileSystemView(metaClient) try { - fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().getTimestamp) + fsView.getAllLatestFileSlicesBeforeOrOn(metaClient.getActiveTimeline.lastInstant().get().requestedTime) .values() .forEach(JFunction.toJavaConsumer[java.util.stream.Stream[FileSlice]] (slices => slices.forEach(JFunction.toJavaConsumer[FileSlice]( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala index 5f8f2673bc1b..c9897588e0b7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestAlterTableDropPartition.scala @@ -44,7 +44,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { // And available public methods does not allow to specify exact instant to get schema from, only latest after some filtering // which may lead to false positives in test scenarios. val lastInstant = metaClient.getActiveTimeline.getCompletedReplaceTimeline.lastInstant().get() - val commitMetadata = HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline.getInstantDetails(lastInstant).get(), classOf[HoodieCommitMetadata]) + val commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstant, metaClient.getActiveTimeline.getInstantDetails(lastInstant).get(), classOf[HoodieCommitMetadata]) val schemaStr = commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY) val schema = new Schema.Parser().parse(schemaStr) val fields = schema.getFields.asScala.map(_.name()) @@ -505,7 +505,7 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { // check schema val metaClient = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName") val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.lastInstant() - val commitMetadata = HoodieCommitMetadata.fromBytes(metaClient.getActiveTimeline.getInstantDetails( + val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(lastInstant.get(), metaClient.getActiveTimeline.getInstantDetails( lastInstant.get()).get(), classOf[HoodieCommitMetadata]) val schemaStr = commitMetadata.getExtraMetadata.get(HoodieCommitMetadata.SCHEMA_KEY) Assertions.assertFalse(StringUtils.isNullOrEmpty(schemaStr)) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala index de3fa5371292..34226ded9f8b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/ddl/TestSpark3DDL.scala @@ -467,7 +467,7 @@ class TestSpark3DDL extends HoodieSparkSqlTestBase { val metaClient = createMetaClient(spark, basePath) val schema = new TableSchemaResolver(metaClient).getTableInternalSchemaFromCommitMetadata.get() val lastInstant = metaClient.getActiveTimeline.filterCompletedInstants().lastInstant().get() - assert(schema.schemaId() == lastInstant.getTimestamp.toLong) + assert(schema.schemaId() == lastInstant.requestedTime.toLong) if (isDropColumn) { assert(schema.getMaxColumnId == currentMaxColumnId) } else { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestCDCForSparkSQL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestCDCForSparkSQL.scala index 66aec4d397a7..5535400cb667 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestCDCForSparkSQL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestCDCForSparkSQL.scala @@ -77,7 +77,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { spark.sql(s"insert into $tableName values (1, 11, 1000, 'a1'), (2, 12, 1000, 'a2')") assert(spark.sql(s"select _hoodie_file_name from $tableName").distinct().count() == 2) val fgForID1 = spark.sql(s"select _hoodie_file_name from $tableName where id=1").head().get(0) - val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly1 = cdcDataFrame(basePath, commitTime1.toLong - 1) cdcDataOnly1.show(false) assertCDCOpCnt(cdcDataOnly1, 2, 0, 0) @@ -132,13 +132,13 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { val metaClient = createMetaClient(spark, basePath) spark.sql(s"insert into $tableName values (1, 'a1', 11, 1000), (2, 'a2', 12, 1000), (3, 'a3', 13, 1000)") - val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly1 = cdcDataFrame(basePath, commitTime1.toLong - 1) cdcDataOnly1.show(false) assertCDCOpCnt(cdcDataOnly1, 3, 0, 0) spark.sql(s"insert into $tableName values (1, 'a1_v2', 11, 1100)") - val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime // here we use `commitTime1` to query the change data in commit 2. // because `commitTime2` is maybe the ts of the compaction operation, not the write operation. val cdcDataOnly2 = cdcDataFrame(basePath, commitTime1.toLong) @@ -162,13 +162,13 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { checkAnswer(change2)(Seq("u", 1, "a1", 11, "a1_v2", 11)) spark.sql(s"update $tableName set name = 'a2_v2', ts = 1200 where id = 2") - val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly3 = cdcDataFrame(basePath, commitTime2.toLong) cdcDataOnly3.show(false) assertCDCOpCnt(cdcDataOnly3, 0, 1, 0) spark.sql(s"delete from $tableName where id = 3") - val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly4 = cdcDataFrame(basePath, commitTime3.toLong) cdcDataOnly4.show(false) assertCDCOpCnt(cdcDataOnly4, 0, 0, 1) @@ -187,7 +187,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { | when matched then update set id = s0.id, name = s0.name, price = s0.price, ts = s0.ts | when not matched then insert * """.stripMargin) - val commitTime5 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime5 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly5 = cdcDataFrame(basePath, commitTime4.toLong) cdcDataOnly5.show(false) assertCDCOpCnt(cdcDataOnly5, 1, 1, 0) @@ -260,19 +260,19 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { | (2, 'a2', 12, 1000, '2022'), | (3, 'a3', 13, 1000, '2022') """.stripMargin) - val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime1 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly1 = cdcDataFrame(basePath, commitTime1.toLong - 1) cdcDataOnly1.show(false) assertCDCOpCnt(cdcDataOnly1, 3, 0, 0) spark.sql(s"insert overwrite table $tableName partition (pt = '2021') values (1, 'a1_v2', 11, 1100)") - val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime2 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly2 = cdcDataFrame(basePath, commitTime2.toLong - 1) cdcDataOnly2.show(false) assertCDCOpCnt(cdcDataOnly2, 1, 0, 1) spark.sql(s"update $tableName set name = 'a2_v2', ts = 1200 where id = 2") - val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime3 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly3 = cdcDataFrame(basePath, commitTime3.toLong - 1) cdcDataOnly3.show(false) assertCDCOpCnt(cdcDataOnly3, 0, 1, 0) @@ -291,7 +291,7 @@ class TestCDCForSparkSQL extends HoodieSparkSqlTestBase { | when matched then update set id = s0.id, name = s0.name, price = s0.price, ts = s0.ts, pt = s0.pt | when not matched then insert * """.stripMargin) - val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().getTimestamp + val commitTime4 = metaClient.reloadActiveTimeline.lastInstant().get().requestedTime val cdcDataOnly4 = cdcDataFrame(basePath, commitTime4.toLong - 1) cdcDataOnly4.show(false) assertCDCOpCnt(cdcDataOnly4, 1, 1, 0) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestTimeTravelTable.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestTimeTravelTable.scala index 972d246e5743..1f40de1777cd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestTimeTravelTable.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/dml/TestTimeTravelTable.scala @@ -49,7 +49,7 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { val metaClient1 = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName1") val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime // 2nd commit instant spark.sql(s"insert into $tableName1 values(1, 'a2', 20, 2000)") @@ -95,7 +95,7 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { val metaClient1 = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName1") val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime val tableName2 = generateTableName @@ -206,10 +206,10 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { val metaClient2 = createMetaClient(spark, path2) val instant1 = metaClient1.getActiveTimeline.getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime val instant2 = metaClient2.getActiveTimeline.getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime val sql = s""" @@ -266,7 +266,7 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { val metaClient = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName") val instant1 = metaClient.getActiveTimeline.getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime // 2nd commit instant spark.sql(s"insert into $tableName values(1, 'a2', 20, 2000)") @@ -309,13 +309,13 @@ class TestTimeTravelTable extends HoodieSparkSqlTestBase { val metaClient = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName") val instant1 = metaClient.reloadActiveTimeline().getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime // add column spark.sql(s"alter table $tableName add columns (company string)") spark.sql(s"insert into $tableName values(2, 'a2', 11, 1100, 'hudi')") val instant2 = metaClient.reloadActiveTimeline().getAllCommitsTimeline - .lastInstant().get().getTimestamp + .lastInstant().get().requestedTime // drop column spark.sql(s"alter table $tableName drop column price") diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala index da080d23c456..6a43205128e7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala @@ -22,13 +22,12 @@ package org.apache.spark.sql.hudi.procedure import org.apache.hudi.DataSourceWriteOptions.{OPERATION, RECORDKEY_FIELD} import org.apache.hudi.common.config.HoodieMetadataConfig import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.testutils.HoodieTestUtils import org.apache.hudi.common.util.collection.Pair import org.apache.hudi.common.util.{Option => HOption} import org.apache.hudi.storage.hadoop.HadoopStorageConfiguration import org.apache.hudi.{DataSourceReadOptions, HoodieCLIUtils, HoodieDataSourceHelpers, HoodieFileIndex} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} @@ -36,7 +35,6 @@ import org.apache.spark.sql.types.{DataTypes, Metadata, StringType, StructField, import org.apache.spark.sql.{Dataset, Row} import java.util - import scala.collection.JavaConverters.asScalaIteratorConverter class TestClusteringProcedure extends HoodieSparkProcedureTestBase { @@ -283,7 +281,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { // The latest clustering should contain 2 file groups val clusteringInstant = clusteringInstants.last - val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.requestedTime) assertResult(true)(clusteringPlan.isPresent) assertResult(2)(clusteringPlan.get().getInputGroups.size()) assertResult(resultA(0)(1))(clusteringPlan.get().getInputGroups.size()) @@ -329,7 +327,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { // The latest clustering should contain 4 file groups(1002,1003,1004,1005) val clusteringInstant = clusteringInstants.last - val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.requestedTime) assertResult(true)(clusteringPlan.isPresent) assertResult(4)(clusteringPlan.get().getInputGroups.size()) @@ -379,7 +377,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { // The latest clustering should contain 3 file groups(1006,1007,1009) val clusteringInstant = clusteringInstants.last - val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.getTimestamp) + val clusteringPlan = HoodieDataSourceHelpers.getClusteringPlan(fs, basePath, clusteringInstant.requestedTime) assertResult(true)(clusteringPlan.isPresent) assertResult(3)(clusteringPlan.get().getInputGroups.size()) @@ -446,7 +444,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { val conf = new Configuration val metaClient = HoodieTestUtils.createMetaClient(new HadoopStorageConfiguration(conf), basePath) - val instants = metaClient.getActiveTimeline.filterPendingClusteringTimeline().getInstants.iterator().asScala.map(_.getTimestamp).toSeq + val instants = metaClient.getActiveTimeline.filterPendingClusteringTimeline().getInstants.iterator().asScala.map(_.requestedTime).toSeq assert(2 == instants.size) checkExceptionContain( @@ -476,7 +474,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { // test with operator execute spark.sql(s"call run_clustering(table => '$tableName', op => 'schedule')") metaClient.reloadActiveTimeline() - val instants2 = metaClient.getActiveTimeline.filterPendingClusteringTimeline().getInstants.iterator().asScala.map(_.getTimestamp).toSeq + val instants2 = metaClient.getActiveTimeline.filterPendingClusteringTimeline().getInstants.iterator().asScala.map(_.requestedTime).toSeq spark.sql(s"call run_clustering(table => '$tableName', instants => '${instants2.mkString(",")}', op => 'execute')") metaClient.reloadActiveTimeline() assert(3 == metaClient.getActiveTimeline.getCompletedReplaceTimeline.getInstants.size()) @@ -779,8 +777,10 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { def avgRecord(commitTimeline: HoodieTimeline): Long = { var totalByteSize = 0L var totalRecordsCount = 0L + val layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion) commitTimeline.getReverseOrderedInstants.toArray.foreach(instant => { - val commitMetadata = HoodieCommitMetadata.fromBytes(commitTimeline.getInstantDetails(instant.asInstanceOf[HoodieInstant]).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(instant.asInstanceOf[HoodieInstant], + commitTimeline.getInstantDetails(instant.asInstanceOf[HoodieInstant]).get, classOf[HoodieCommitMetadata]) totalByteSize = totalByteSize + commitMetadata.fetchTotalBytesWritten() totalRecordsCount = totalRecordsCount + commitMetadata.fetchTotalRecordsWritten() }) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala index 470d49c03b11..07ff3ed86afd 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestCompactionProcedure.scala @@ -290,7 +290,7 @@ class TestCompactionProcedure extends HoodieSparkProcedureTestBase { val metaClient = createMetaClient(tmp.getCanonicalPath) val instants = metaClient.getActiveTimeline.filterPendingCompactionTimeline().getInstants assertResult(1)(instants.size()) - val ts = instants.get(0).getTimestamp + val ts = instants.get(0).requestedTime assertResult(1)(spark.sql(s"call run_compaction(table => '$tableName', op => 'execute', instants => '$ts')").collect().length) checkExceptionContain( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala index e85d660f0f52..6fc260da9ef9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestHdfsParquetImportProcedure.scala @@ -18,13 +18,12 @@ package org.apache.spark.sql.hudi.procedure import org.apache.hudi.common.model.HoodieTableType -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline +import org.apache.hudi.common.table.timeline.TimelineUtils import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, HoodieTestUtils} import org.apache.hudi.common.util.StringUtils.getUTF8Bytes import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.storage.{HoodieStorage, HoodieStorageUtils, StoragePath} import org.apache.hudi.testutils.HoodieClientTestUtils - import org.apache.avro.generic.GenericRecord import org.apache.hadoop.fs.Path import org.apache.parquet.avro.AvroParquetWriter @@ -38,7 +37,6 @@ import java.io.IOException import java.util import java.util.Objects import java.util.concurrent.TimeUnit - import scala.collection.JavaConverters._ class TestHdfsParquetImportProcedure extends HoodieSparkProcedureTestBase { @@ -115,7 +113,7 @@ class TestHdfsParquetImportProcedure extends HoodieSparkProcedureTestBase { @throws[IOException] def createInsertRecords(srcFolder: Path): util.List[GenericRecord] = { val srcFile: Path = new Path(srcFolder.toString, "file1.parquet") - val startTime: Long = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime / 1000 + val startTime: Long = TimelineUtils.parseDateFromInstantTime("20170203000000").getTime / 1000 val records: util.List[GenericRecord] = new util.ArrayList[GenericRecord] for (recordNum <- 0 until 96) { records.add(new HoodieTestDataGenerator().generateGenericRecord(recordNum.toString, @@ -138,7 +136,7 @@ class TestHdfsParquetImportProcedure extends HoodieSparkProcedureTestBase { @throws[IOException] def createUpsertRecords(srcFolder: Path): util.List[GenericRecord] = { val srcFile = new Path(srcFolder.toString, "file1.parquet") - val startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime / 1000 + val startTime = TimelineUtils.parseDateFromInstantTime("20170203000000").getTime / 1000 val records = new util.ArrayList[GenericRecord] // 10 for update val dataGen = new HoodieTestDataGenerator 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 ae33def299a6..bd5deb396406 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 @@ -20,9 +20,9 @@ package org.apache.spark.sql.hudi.procedure import org.apache.hudi.avro.HoodieAvroUtils import org.apache.hudi.common.model.HoodieFileFormat import org.apache.hudi.common.table.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.HoodieTimeline import org.apache.hudi.common.table.view.HoodieTableFileSystemView import org.apache.hudi.common.testutils.{HoodieTestDataGenerator, SchemaTestUtil} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.storage.{StoragePath, StoragePathInfo} import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient @@ -547,7 +547,7 @@ class TestRepairsProcedure extends HoodieSparkProcedureTestBase { @throws[IOException] def createEmptyCleanRequestedFile(basePath: String, instantTime: String, configuration: Configuration): Unit = { - val commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + HoodieTimeline.makeRequestedCleanerFileName(instantTime)) + val commitFilePath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + INSTANT_FILE_NAME_GENERATOR.makeRequestedCleanerFileName(instantTime)) val fs = HadoopFSUtils.getFs(basePath, configuration) val os = fs.create(commitFilePath, true) os.close() diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala index e2923f3be86b..255e02b3e526 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestRunRollbackInflightTableServiceProcedure.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hudi.procedure import org.apache.hudi.common.table.HoodieTableMetaClient import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.junit.jupiter.api.Assertions @@ -56,13 +57,13 @@ class TestRunRollbackInflightTableServiceProcedure extends HoodieSparkProcedureT val clusteringInstant = metaClient.getActiveTimeline.getCompletedReplaceTimeline.getInstants.get(0) metaClient.getActiveTimeline.deleteInstantFileIfExists(clusteringInstant) - val clusteringInstantTime = clusteringInstant.getTimestamp + val clusteringInstantTime = clusteringInstant.requestedTime spark.sql(s"call run_rollback_inflight_tableservice(table => '$tableName', pending_instant => '$clusteringInstantTime')") Assertions.assertTrue(!metaClient.reloadActiveTimeline().getInstants - .contains(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime))) + .contains(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime))) Assertions.assertTrue(metaClient.reloadActiveTimeline().getInstants - .contains(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime))) + .contains(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusteringInstantTime))) }} } @@ -107,13 +108,13 @@ class TestRunRollbackInflightTableServiceProcedure extends HoodieSparkProcedureT val compactionInstant: HoodieInstant = metaClient.getActiveTimeline.getReverseOrderedInstants.findFirst().get() metaClient.getActiveTimeline.deleteInstantFileIfExists(compactionInstant) - val compactionInstantTime = compactionInstant.getTimestamp + val compactionInstantTime = compactionInstant.requestedTime spark.sql(s"call run_rollback_inflight_tableservice(table => '$tableName', pending_instant => '$compactionInstantTime', delete_request_instant_file => true)") Assertions.assertTrue(!metaClient.reloadActiveTimeline().getInstants - .contains(new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime))) + .contains(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime))) Assertions.assertTrue(!metaClient.reloadActiveTimeline().getInstants - .contains(new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime))) + .contains(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionInstantTime))) } } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowFileStatusProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowFileStatusProcedure.scala index 27196082c34d..89d31d4b3647 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowFileStatusProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestShowFileStatusProcedure.scala @@ -83,7 +83,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002, 1000)") checkAnswer(s"call show_file_status(table => '$tableName', partition => '$partition', file => '$firstCleanedDataFile')")( - Seq(FileStatus.DELETED.toString, HoodieTimeline.CLEAN_ACTION, metaClient.reloadActiveTimeline().getCleanerTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE) + Seq(FileStatus.DELETED.toString, HoodieTimeline.CLEAN_ACTION, metaClient.reloadActiveTimeline().getCleanerTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE) ) spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002, 1001)") @@ -113,7 +113,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { cleanedDataFile = getAnyOneDataFile(before, after) checkAnswer(s"call show_file_status(table => '$tableName', partition => '$partition', file => '${cleanedDataFile.get}')")( - Seq(FileStatus.DELETED.toString, HoodieTimeline.RESTORE_ACTION, metaClient.reloadActiveTimeline().getRestoreTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) + Seq(FileStatus.DELETED.toString, HoodieTimeline.RESTORE_ACTION, metaClient.reloadActiveTimeline().getRestoreTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) val latestTime = HoodieDataSourceHelpers.latestCommit(fs, basePath) spark.sql(s"insert into $tableName values(7, 'a7', 15, 1000, 1000)") @@ -127,7 +127,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { after = getAllDataFile(fs, basePath, Option.apply(partition)) cleanedDataFile = getAnyOneDataFile(before, after) checkAnswer(s"call show_file_status(table => '$tableName', partition => '$partition', file => '${cleanedDataFile.get}')")( - Seq(FileStatus.DELETED.toString, HoodieTimeline.ROLLBACK_ACTION, metaClient.reloadActiveTimeline().getRollbackTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) + Seq(FileStatus.DELETED.toString, HoodieTimeline.ROLLBACK_ACTION, metaClient.reloadActiveTimeline().getRollbackTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) // unknown checkAnswer(s"call show_file_status(table => '$tableName', partition => '$partition', file => 'unknown')")( @@ -187,7 +187,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002, 1000)") checkAnswer(s"call show_file_status(table => '$tableName', file => '$firstCleanedDataFile')")( - Seq(FileStatus.DELETED.toString, HoodieTimeline.CLEAN_ACTION, metaClient.reloadActiveTimeline().getCleanerTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE) + Seq(FileStatus.DELETED.toString, HoodieTimeline.CLEAN_ACTION, metaClient.reloadActiveTimeline().getCleanerTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE) ) spark.sql(s"insert into $tableName values(3, 'a3', 10, 1002, 1000)") @@ -217,7 +217,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { cleanedDataFile = getAnyOneDataFile(before, after) checkAnswer(s"call show_file_status(table => '$tableName', file => '${cleanedDataFile.get}')")( Seq(FileStatus.DELETED.toString, HoodieTimeline.RESTORE_ACTION, - metaClient.reloadActiveTimeline().getRestoreTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) + metaClient.reloadActiveTimeline().getRestoreTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) val latestTime = HoodieDataSourceHelpers.latestCommit(fs, basePath) spark.sql(s"insert into $tableName values(7, 'a7', 15, 1000, 1000)") @@ -231,7 +231,7 @@ class TestShowFileStatusProcedure extends HoodieSparkProcedureTestBase { after = getAllDataFile(fs, basePath, partition) cleanedDataFile = getAnyOneDataFile(before, after) checkAnswer(s"call show_file_status(table => '$tableName', file => '${cleanedDataFile.get}')")( - Seq(FileStatus.DELETED.toString, HoodieTimeline.ROLLBACK_ACTION, metaClient.reloadActiveTimeline().getRollbackTimeline.lastInstant().get.getTimestamp, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) + Seq(FileStatus.DELETED.toString, HoodieTimeline.ROLLBACK_ACTION, metaClient.reloadActiveTimeline().getRollbackTimeline.lastInstant().get.requestedTime, TimelineType.ACTIVE.toString, DEFAULT_VALUE)) // unknown checkAnswer(s"call show_file_status(table => '$tableName', file => 'unknown')")( diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala index fed1ea7c2c8f..f3e65a84bbfe 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestUpgradeOrDowngradeProcedure.scala @@ -21,6 +21,7 @@ import org.apache.hudi.common.config.HoodieConfig import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, HoodieTableVersion} import org.apache.hudi.common.util.{BinaryUtil, ConfigUtils, StringUtils} import org.apache.hudi.storage.StoragePath +import org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR import org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient import org.apache.spark.sql.hudi.common.HoodieSparkSqlTestBase.NAME_FORMAT_0_X @@ -187,7 +188,7 @@ class TestUpgradeOrDowngradeProcedure extends HoodieSparkProcedureTestBase { metaClient.getTableConfig.getTableVersion.versionCode() } // Verify whether the naming format of instant files is consistent with 0.x - metaClient.reloadActiveTimeline().getInstants.iterator().asScala.forall(f => NAME_FORMAT_0_X.matcher(f.getFileName).find()) + metaClient.reloadActiveTimeline().getInstants.iterator().asScala.forall(f => NAME_FORMAT_0_X.matcher(INSTANT_FILE_NAME_GENERATOR.getFileName(f)).find()) checkAnswer(s"select id, name, price, ts from $tableName")( Seq(1, "a1", 10.0, 1000) ) diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala index 717e7974c95c..cbff3a052f6f 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark3ParquetSchemaEvolutionUtils.scala @@ -27,9 +27,10 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.InternalSchemaUtils import org.apache.hudi.storage.hadoop.HoodieHadoopStorage - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path +import org.apache.hudi.common.table.timeline.TimelineLayout +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.parquet.hadoop.metadata.FileMetaData import org.apache.spark.sql.HoodieSchemaUtils import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection @@ -39,7 +40,6 @@ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType} import java.time.ZoneId - import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, @@ -57,9 +57,12 @@ class Spark3ParquetSchemaEvolutionUtils(sharedConf: Configuration, private lazy val tablePath: String = sharedConf.get(SparkInternalSchemaConverter.HOODIE_TABLE_PATH) private lazy val fileSchema: InternalSchema = if (shouldUseInternalSchema) { val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; + //TODO: HARDCODED TIMELINE OBJECT val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + val layout = TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION) InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, - new HoodieHadoopStorage(tablePath, sharedConf), if (validCommits == null) "" else validCommits) + new HoodieHadoopStorage(tablePath, sharedConf), if (validCommits == null) "" else validCommits, + layout.getInstantFileNameParser, layout.getCommitMetadataSerDe, layout.getInstantGenerator) } else { null } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala index 6887c53b1734..cc61a55bd3bf 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/command/AlterTableCommand.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hudi.command import org.apache.hudi.avro.AvroSchemaUtils.getAvroRecordQualifiedName import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.model.{HoodieCommitMetadata, WriteOperationType} -import org.apache.hudi.common.table.timeline.HoodieInstant import org.apache.hudi.common.table.timeline.HoodieInstant.State import org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} @@ -271,10 +270,11 @@ object AlterTableCommand extends Logging { val hoodieTable = HoodieSparkTable.create(client.getConfig, client.getEngineContext) val timeLine = hoodieTable.getActiveTimeline - val requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime) + val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator + val requested = instantGenerator.createNewInstant(State.REQUESTED, commitActionType, instantTime) val metadata = new HoodieCommitMetadata metadata.setOperationType(WriteOperationType.ALTER_SCHEMA) - timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(metadata)) + timeLine.transitionRequestedToInflight(requested, serializeCommitMetadata(metaClient.getTimelineLayout.getCommitMetadataSerDe, metadata)) val extraMeta = new util.HashMap[String, String]() extraMeta.put(SerDeHelper.LATEST_SCHEMA, SerDeHelper.toJson(internalSchema.setSchemaId(instantTime.toLong))) val schemaManager = new FileBasedInternalSchemaStorageManager(metaClient) diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala index 9cb393f15467..36f7350496a1 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark33LegacyHoodieParquetFileFormat.scala @@ -28,12 +28,13 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.storage.hadoop.HoodieHadoopStorage - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hudi.common.table.timeline.TimelineLayout +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -52,7 +53,6 @@ import org.apache.spark.sql.types.{AtomicType, DataType, StructField, StructType import org.apache.spark.util.SerializableConfiguration import java.net.URI - import scala.collection.convert.ImplicitConversions.`collection AsScalaIterable` /** @@ -161,9 +161,12 @@ class Spark33LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val fileSchema = if (shouldUseInternalSchema) { val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + //TODO: HARDCODED TIMELINE OBJECT + val layout = TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION) val storage = new HoodieHadoopStorage(tablePath, sharedConf) InternalSchemaCache.getInternalSchemaByVersionId( - commitInstantTime, tablePath, storage, if (validCommits == null) "" else validCommits) + commitInstantTime, tablePath, storage, if (validCommits == null) "" else validCommits, + layout.getInstantFileNameParser, layout.getCommitMetadataSerDe, layout.getInstantGenerator) } else { null } diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3.3.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java index 614e27a657a5..c9de99dc5823 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; @@ -195,7 +196,8 @@ public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exce dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); metaClient.reloadActiveTimeline(); - Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields); + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, + populateMetaFields, INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); @@ -243,7 +245,7 @@ public void testLargeWrites(boolean populateMetaFields) throws Exception { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, - populateMetaFields); + populateMetaFields, INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala index 679c290b1f3a..b9f8b9eedecf 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark34LegacyHoodieParquetFileFormat.scala @@ -26,11 +26,12 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.storage.hadoop.HoodieHadoopStorage - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hudi.common.table.timeline.TimelineLayout +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -172,8 +173,12 @@ class Spark34LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val fileSchema = if (shouldUseInternalSchema) { val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) + //TODO: HARDCODED TIMELINE OBJECT + val layout = TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION) val storage = new HoodieHadoopStorage(tablePath, sharedConf) - InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, storage, if (validCommits == null) "" else validCommits) + InternalSchemaCache.getInternalSchemaByVersionId(commitInstantTime, tablePath, storage, + if (validCommits == null) "" else validCommits, + layout.getInstantFileNameParser, layout.getCommitMetadataSerDe, layout.getInstantGenerator) } else { null } diff --git a/hudi-spark-datasource/hudi-spark3.4.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3.4.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java index 99ccd7d03082..ec0cba2a0b79 100644 --- a/hudi-spark-datasource/hudi-spark3.4.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3.4.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java @@ -46,6 +46,7 @@ import java.util.Map; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.testutils.SparkDatasetTestUtils.ENCODER; import static org.apache.hudi.testutils.SparkDatasetTestUtils.STRUCT_TYPE; import static org.apache.hudi.testutils.SparkDatasetTestUtils.getRandomRows; @@ -193,7 +194,8 @@ public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exce dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); metaClient.reloadActiveTimeline(); - Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields); + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), + instantTime, populateMetaFields, INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); @@ -241,7 +243,7 @@ public void testLargeWrites(boolean populateMetaFields) throws Exception { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, - populateMetaFields); + populateMetaFields, INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala index dd0296bee93a..b4e6896408b3 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/Spark35LegacyHoodieParquetFileFormat.scala @@ -26,11 +26,12 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.action.InternalSchemaMerger import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper} import org.apache.hudi.storage.hadoop.HoodieHadoopStorage - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hudi.common.table.timeline.TimelineLayout +import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion import org.apache.parquet.filter2.compat.FilterCompat import org.apache.parquet.filter2.predicate.FilterApi import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS @@ -174,8 +175,11 @@ class Spark35LegacyHoodieParquetFileFormat(private val shouldAppendPartitionValu val commitInstantTime = FSUtils.getCommitTime(filePath.getName).toLong; val validCommits = sharedConf.get(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST) val storage = new HoodieHadoopStorage(tablePath, sharedConf) + //TODO: HARDCODED TIMELINE OBJECT + val layout = TimelineLayout.fromVersion(TimelineLayoutVersion.CURR_LAYOUT_VERSION) InternalSchemaCache.getInternalSchemaByVersionId( - commitInstantTime, tablePath, storage, if (validCommits == null) "" else validCommits) + commitInstantTime, tablePath, storage, if (validCommits == null) "" else validCommits, + layout.getInstantFileNameParser, layout.getCommitMetadataSerDe, layout.getInstantGenerator) } else { null } diff --git a/hudi-spark-datasource/hudi-spark3.5.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java b/hudi-spark-datasource/hudi-spark3.5.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java index 99ccd7d03082..b58decad8dec 100644 --- a/hudi-spark-datasource/hudi-spark3.5.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java +++ b/hudi-spark-datasource/hudi-spark3.5.x/src/test/java/org/apache/hudi/spark3/internal/TestHoodieDataSourceInternalBatchWrite.java @@ -21,6 +21,7 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; +import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.internal.HoodieBulkInsertInternalWriterTestBase; @@ -193,7 +194,7 @@ public void testMultipleDataSourceWrites(boolean populateMetaFields) throws Exce dataSourceInternalBatchWrite.commit(commitMessages.toArray(new HoodieWriterCommitMessage[0])); metaClient.reloadActiveTimeline(); - Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields); + Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, populateMetaFields, HoodieTestUtils.INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); @@ -241,7 +242,7 @@ public void testLargeWrites(boolean populateMetaFields) throws Exception { metaClient.reloadActiveTimeline(); Dataset result = HoodieClientTestUtils.readCommit(basePath, sqlContext, metaClient.getCommitTimeline(), instantTime, - populateMetaFields); + populateMetaFields, HoodieTestUtils.INSTANT_GENERATOR); // verify output assertOutput(totalInputRows, result, instantTime, Option.empty(), populateMetaFields); diff --git a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java index 8d3e2ce56d2a..76932b25c2f0 100644 --- a/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java +++ b/hudi-sync/hudi-adb-sync/src/main/java/org/apache/hudi/sync/adb/HoodieAdbJdbcClient.java @@ -262,7 +262,7 @@ public Option getLastCommitTimeSynced(String tableName) { @Override public void updateLastCommitTimeSynced(String tableName) { // Set the last commit time from the TBLProperties - String lastCommitSynced = getActiveTimeline().lastInstant().get().getTimestamp(); + String lastCommitSynced = getActiveTimeline().lastInstant().get().requestedTime(); try { String sql = constructUpdateTblPropertiesSql(tableName, lastCommitSynced); executeAdbSql(sql); diff --git a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java index c116393d1f4a..ac3f474d6769 100644 --- a/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java +++ b/hudi-sync/hudi-datahub-sync/src/main/java/org/apache/hudi/sync/datahub/DataHubSyncClient.java @@ -78,7 +78,7 @@ public Option getLastCommitTimeSynced(String tableName) { @Override public void updateLastCommitTimeSynced(String tableName) { - updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, getActiveTimeline().lastInstant().get().getTimestamp())); + updateTableProperties(tableName, Collections.singletonMap(HOODIE_LAST_COMMIT_TIME_SYNC, getActiveTimeline().lastInstant().get().requestedTime())); } @Override 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 29db31db2c74..bd00773e9138 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 @@ -356,7 +356,7 @@ public Option getLastReplicatedTime(String tableName) { } public void updateLastReplicatedTimeStamp(String tableName, String timeStamp) { - if (getActiveTimeline().getInstantsAsStream().noneMatch(i -> i.getTimestamp().equals(timeStamp))) { + if (getActiveTimeline().getInstantsAsStream().noneMatch(i -> i.requestedTime().equals(timeStamp))) { throw new HoodieHiveSyncException( "Not a valid completed timestamp " + timeStamp + " for table " + tableName); } @@ -403,7 +403,7 @@ public void close() { public void updateLastCommitTimeSynced(String tableName) { // Set the last commit time and commit completion from the TBLproperties HoodieTimeline activeTimeline = getActiveTimeline(); - Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::getTimestamp); + Option lastCommitSynced = activeTimeline.lastInstant().map(HoodieInstant::requestedTime); Option lastCommitCompletionSynced = activeTimeline .getInstantsOrderedByCompletionTime() .skip(activeTimeline.countInstants() - 1) 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 9c85fab18cc4..9db372e4b2e4 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 @@ -90,6 +90,7 @@ import static org.apache.hudi.common.table.timeline.HoodieTimeline.COMMIT_ACTION; import static org.apache.hudi.common.table.timeline.HoodieTimeline.DELTA_COMMIT_ACTION; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.hadoop.fs.HadoopFSUtils.getRelativePartitionPath; import static org.apache.hudi.hive.HiveSyncConfig.HIVE_SYNC_FILTER_PUSHDOWN_ENABLED; import static org.apache.hudi.hive.HiveSyncConfig.RECREATE_HIVE_TABLE_ON_ERROR; @@ -463,7 +464,7 @@ public void testBasicSync(boolean useSchemaFromCommitMetadata, String syncMode, assertEquals( Arrays.asList("400", "700", "800"), metaClient.getActiveTimeline().getInstants().stream() - .map(HoodieInstant::getTimestamp).sorted() + .map(HoodieInstant::requestedTime).sorted() .collect(Collectors.toList())); reInitHiveSyncClient(); @@ -1780,9 +1781,9 @@ 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().getInstantsAsStream() - .filter(inst -> inst.getTimestamp().equals(commitTime2)) - .findFirst().get().getFileName()); + + INSTANT_FILE_NAME_GENERATOR.getFileName(hiveClient.getActiveTimeline().getInstantsAsStream() + .filter(inst -> inst.requestedTime().equals(commitTime2)) + .findFirst().get())); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); try { @@ -1824,9 +1825,9 @@ public void testNotPickingOlderParquetFileWhenLatestCommitReadFailsForExistingTa reInitHiveSyncClient(); // now delete the evolved commit instant Path fullPath = new Path(HiveTestUtil.basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + hiveClient.getActiveTimeline().getInstantsAsStream() - .filter(inst -> inst.getTimestamp().equals(commitTime2)) - .findFirst().get().getFileName()); + + INSTANT_FILE_NAME_GENERATOR.getFileName(hiveClient.getActiveTimeline().getInstantsAsStream() + .filter(inst -> inst.requestedTime().equals(commitTime2)) + .findFirst().get())); assertTrue(HiveTestUtil.fileSystem.delete(fullPath, false)); try { reSyncHiveTable(); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java index fbf4ff522d55..9546694ed4d3 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestCluster.java @@ -29,7 +29,6 @@ import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.InProcessTimeGenerator; import org.apache.hudi.common.testutils.NetworkTestUtils; import org.apache.hudi.common.testutils.SchemaTestUtil; @@ -76,6 +75,8 @@ import java.util.Properties; import java.util.UUID; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.junit.jupiter.api.Assertions.fail; @@ -173,9 +174,9 @@ public void createCOWTable(String commitTime, int numberOfPartitions, String dbN } private void createCommitFile(HoodieCommitMetadata commitMetadata, String commitTime, String basePath) throws IOException { - byte[] bytes = serializeCommitMetadata(commitMetadata).get(); + byte[] bytes = serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get(); Path fullPath = new Path(basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" - + HoodieTimeline.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime())); + + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(commitTime + "_" + InProcessTimeGenerator.createNewInstantTime())); OutputStream fsout = dfsCluster.getFileSystem().create(fullPath, true); fsout.write(bytes); fsout.close(); diff --git a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java index 309c55236cdf..979a6e9d29e5 100644 --- a/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java +++ b/hudi-sync/hudi-hive-sync/src/test/java/org/apache/hudi/hive/testutils/HiveTestUtil.java @@ -43,7 +43,6 @@ import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.testutils.FileCreateUtils; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.common.testutils.InProcessTimeGenerator; @@ -104,6 +103,8 @@ import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeRollbackMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_BATCH_SYNC_PARTITION_NUM; import static org.apache.hudi.hive.HiveSyncConfigHolder.HIVE_PASS; @@ -365,15 +366,15 @@ public static void addRollbackInstantToTable(String instantTime, String commitTo createMetaFile( basePath, - HoodieTimeline.makeRequestedRollbackFileName(instantTime), + INSTANT_FILE_NAME_GENERATOR.makeRequestedRollbackFileName(instantTime), getUTF8Bytes("")); createMetaFile( basePath, - HoodieTimeline.makeInflightRollbackFileName(instantTime), + INSTANT_FILE_NAME_GENERATOR.makeInflightRollbackFileName(instantTime), getUTF8Bytes("")); createMetaFile( basePath, - HoodieTimeline.makeRollbackFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + INSTANT_FILE_NAME_GENERATOR.makeRollbackFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), serializeRollbackMetadata(rollbackMetadata).get()); } @@ -772,15 +773,15 @@ private static void checkResult(boolean result) { public static void createCommitFile(HoodieCommitMetadata commitMetadata, String instantTime, String basePath) throws IOException { createMetaFile( basePath, - HoodieTimeline.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - serializeCommitMetadata(commitMetadata).get()); + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } public static void createReplaceCommitFile(HoodieReplaceCommitMetadata commitMetadata, String instantTime) throws IOException { createMetaFile( basePath, - HoodieTimeline.makeReplaceFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - serializeCommitMetadata(commitMetadata).get()); + INSTANT_FILE_NAME_GENERATOR.makeReplaceFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } public static void createCommitFileWithSchema(HoodieCommitMetadata commitMetadata, String instantTime, boolean isSimpleSchema) throws IOException { @@ -792,16 +793,16 @@ private static void createCompactionCommitFile(HoodieCommitMetadata commitMetada throws IOException { createMetaFile( basePath, - HoodieTimeline.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - serializeCommitMetadata(commitMetadata).get()); + INSTANT_FILE_NAME_GENERATOR.makeCommitFileName(instantTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get()); } private static void createDeltaCommitFile(HoodieCommitMetadata deltaCommitMetadata, String deltaCommitTime) throws IOException { createMetaFile( basePath, - HoodieTimeline.makeDeltaFileName(deltaCommitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), - serializeCommitMetadata(deltaCommitMetadata).get()); + INSTANT_FILE_NAME_GENERATOR.makeDeltaFileName(deltaCommitTime + "_" + InProcessTimeGenerator.createNewInstantTime()), + serializeCommitMetadata(COMMIT_METADATA_SER_DE, deltaCommitMetadata).get()); } private static void createMetaFile(String basePath, String fileName, byte[] bytes) 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 4c136041b8fe..ae71c7119869 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 @@ -694,7 +694,7 @@ private boolean shouldThrowExceptionIfLocalViewBehind(HoodieTimeline localTimeli // When performing async clean, we may have one more .clean.completed after lastInstantTs. // In this case, we do not need to throw an exception. return !lastInstant.isPresent() || !lastInstant.get().getAction().equals(HoodieTimeline.CLEAN_ACTION) - || !localTimeline.findInstantsBefore(lastInstant.get().getTimestamp()).getTimelineHash().equals(timelineHashFromClient); + || !localTimeline.findInstantsBefore(lastInstant.get().requestedTime()).getTimelineHash().equals(timelineHashFromClient); } private boolean isRefreshCheckDisabledInQuery(Context ctx) { diff --git a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java index 9f62f04a5e91..7588518cd322 100644 --- a/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java +++ b/hudi-timeline-service/src/test/java/org/apache/hudi/timeline/service/handlers/marker/TestMarkerBasedEarlyConflictDetectionRunnable.java @@ -48,6 +48,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.getDefaultStorageConf; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; @@ -106,7 +107,7 @@ public void testMarkerConflictDetectionRunnable() throws IOException, Interrupte prepareFiles(rootBaseMarkerDir, currentInstantTime, currentMarkers, storage); HashSet oldInstants = new HashSet<>(); - oldInstants.add(new HoodieInstant(false, "commit", oldInstant)); + oldInstants.add(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", oldInstant)); when(markerHandler.getAllMarkers(currentMarkerDir)).thenReturn(currentMarkers); ScheduledExecutorService detectorExecutor = Executors.newSingleThreadScheduledExecutor(); 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 1784a54209a1..9c0026bdafb8 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 @@ -287,7 +287,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO Option lastCommit = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { - return lastCommit.get().getTimestamp(); + return lastCommit.get().requestedTime(); } return "0"; } @@ -319,7 +319,7 @@ private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation .setConf(HadoopFSUtils.getStorageConfWithCopy(fs.getConf())) .setBasePath(sourceTableLocation).build(); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstantsAsStream().map(HoodieInstant::getTimestamp) + .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstantsAsStream().map(HoodieInstant::requestedTime) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { LOG.warn( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java index 99ce6e9e60c7..b424a4f1d961 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieClusteringJob.java @@ -23,8 +23,8 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.config.HoodieCleanConfig; @@ -217,7 +217,7 @@ private int doCluster(JavaSparkContext jsc) throws Exception { Option firstClusteringInstant = metaClient.getActiveTimeline().getFirstPendingClusterInstant(); if (firstClusteringInstant.isPresent()) { - cfg.clusteringInstantTime = firstClusteringInstant.get().getTimestamp(); + cfg.clusteringInstantTime = firstClusteringInstant.get().requestedTime(); LOG.info("Found the earliest scheduled clustering instant which will be executed: " + cfg.clusteringInstantTime); } else { @@ -266,11 +266,11 @@ private int doScheduleAndCluster(JavaSparkContext jsc) throws Exception { if (lastClusterOpt.isPresent()) { HoodieInstant inflightClusteringInstant = lastClusterOpt.get(); - Date clusteringStartTime = HoodieActiveTimeline.parseDateFromInstantTime(inflightClusteringInstant.getTimestamp()); + Date clusteringStartTime = TimelineUtils.parseDateFromInstantTime(inflightClusteringInstant.requestedTime()); if (clusteringStartTime.getTime() + cfg.maxProcessingTimeMs < System.currentTimeMillis()) { // if there has failed clustering, then we will use the failed clustering instant-time to trigger next clustering action which will rollback and clustering. LOG.info("Found failed clustering instant at : " + inflightClusteringInstant + "; Will rollback the failed clustering and re-trigger again."); - instantTime = Option.of(inflightClusteringInstant.getTimestamp()); + instantTime = Option.of(inflightClusteringInstant.requestedTime()); } else { LOG.info(inflightClusteringInstant + " might still be in progress, will trigger a new clustering job."); } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java index ae42e8677b59..4cf59d58e2e1 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieCompactor.java @@ -261,7 +261,7 @@ private int doCompact(JavaSparkContext jsc) throws Exception { metaClient = HoodieTableMetaClient.reload(metaClient); Option firstCompactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); if (firstCompactionInstant.isPresent()) { - cfg.compactionInstantTime = firstCompactionInstant.get().getTimestamp(); + cfg.compactionInstantTime = firstCompactionInstant.get().requestedTime(); LOG.info("Found the earliest scheduled compaction instant which will be executed: " + cfg.compactionInstantTime); } else { 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 9ff83534ff6a..5d7f9dae90a0 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 @@ -26,7 +26,6 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; @@ -59,6 +58,9 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; + /** * TODO: [HUDI-8294] * A validator with spark-submit to ensure there are no dangling data files in the data table. @@ -306,10 +308,10 @@ public void doDataTableValidation() { HoodieDataTableUtils.getBaseAndLogFilePathsFromFileSystem(tableMetadata, cfg.basePath); // verify that no data files present with commit time < earliest commit in active timeline. if (metaClient.getActiveTimeline().firstInstant().isPresent()) { - String earliestInstant = metaClient.getActiveTimeline().firstInstant().get().getTimestamp(); + String earliestInstant = metaClient.getActiveTimeline().firstInstant().get().requestedTime(); List danglingFilePaths = allDataFilePaths.stream().filter(path -> { String instantTime = FSUtils.getCommitTime(path.getName()); - return HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.LESSER_THAN, + return compareTimestamps(instantTime, LESSER_THAN, earliestInstant); }).collect(Collectors.toList()); @@ -334,7 +336,7 @@ public void doDataTableValidation() { List danglingFiles = engineContext.flatMap(hoodieInstants, instant -> { Option> filesFromTimeline = RepairUtils.getBaseAndLogFilePathsFromTimeline( activeTimeline, instant); - List baseAndLogFilesFromFs = instantToFilesMap.containsKey(instant.getTimestamp()) ? instantToFilesMap.get(instant.getTimestamp()) + List baseAndLogFilesFromFs = instantToFilesMap.containsKey(instant.requestedTime()) ? instantToFilesMap.get(instant.requestedTime()) : Collections.emptyList(); if (!baseAndLogFilesFromFs.isEmpty()) { Set danglingInstantFiles = new HashSet<>(baseAndLogFilesFromFs); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java index 4e6fcf4f669c..f82c8149e364 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieDropPartitionsTool.java @@ -23,10 +23,10 @@ import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieRecordPayload; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; @@ -298,7 +298,7 @@ public void run() { TimeGenerator timeGenerator = TimeGenerators .getTimeGenerator(HoodieTimeGeneratorConfig.defaultConfig(cfg.basePath), HadoopFSUtils.getStorageConf(jsc.hadoopConfiguration())); - cfg.instantTime = HoodieActiveTimeline.createNewInstantTime(true, timeGenerator); + cfg.instantTime = TimelineUtils.generateInstantTime(true, timeGenerator); } LOG.info(cfg.toString()); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java index b68c683c31e6..7071e57c9309 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieIndexer.java @@ -282,7 +282,7 @@ private int runIndexing(JavaSparkContext jsc) throws Exception { .filterPendingIndexTimeline() .firstInstant(); if (earliestPendingIndexInstant.isPresent()) { - cfg.indexInstantTime = earliestPendingIndexInstant.get().getTimestamp(); + cfg.indexInstantTime = earliestPendingIndexInstant.get().requestedTime(); LOG.info("Found the earliest scheduled indexing instant which will be executed: " + cfg.indexInstantTime); } else { 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 dfd49419962b..5bdfa31821cd 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 @@ -126,7 +126,9 @@ import static org.apache.hudi.common.model.HoodieRecord.FILENAME_METADATA_FIELD; import static org.apache.hudi.common.model.HoodieRecord.PARTITION_PATH_METADATA_FIELD; import static org.apache.hudi.common.model.HoodieRecord.RECORD_KEY_METADATA_FIELD; -import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.apache.hudi.io.storage.HoodieSparkIOFactory.getHoodieSparkIOFactory; import static org.apache.hudi.metadata.HoodieTableMetadata.getMetadataTableBasePath; @@ -556,7 +558,8 @@ public boolean doMetadataTableValidation() { 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()); + instant = metaClient.createNewInstant(HoodieInstant.State.REQUESTED, instant.getAction(), + instant.requestedTime()); HoodieCleanerPlan cleanerPlan = CleanerUtils.getCleanerPlan(metaClient, instant); return cleanerPlan.getFilePathsToBeDeletedPerPartition().values().stream().flatMap(cleanerFileInfoList -> @@ -727,7 +730,7 @@ List validatePartitions(HoodieSparkEngineContext engineContext, StorageP if (partitionCreationTimeOpt.isPresent() && !completedTimeline.containsInstant(partitionCreationTimeOpt.get())) { Option lastInstant = completedTimeline.lastInstant(); if (lastInstant.isPresent() - && HoodieTimeline.compareTimestamps(partitionCreationTimeOpt.get(), GREATER_THAN, lastInstant.get().getTimestamp())) { + && compareTimestamps(partitionCreationTimeOpt.get(), GREATER_THAN, lastInstant.get().requestedTime())) { LOG.warn("Ignoring additional partition {}, as it was deduced to be part of a " + "latest completed commit which was inflight when FS based listing was polled.", partitionFromDMT); actualAdditionalPartitionsInMDT.remove(partitionFromDMT); @@ -785,8 +788,8 @@ List getPartitionsFromFileSystem(HoodieEngineContext engineContext, Stor if (!completedTimeline.containsOrBeforeTimelineStarts(instantTime)) { Option lastInstant = completedTimeline.lastInstant(); return lastInstant.isPresent() - && HoodieTimeline.compareTimestamps( - instantTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, lastInstant.get().getTimestamp()); + && compareTimestamps( + instantTime, LESSER_THAN_OR_EQUALS, lastInstant.get().requestedTime()); } return true; } else { @@ -1089,7 +1092,7 @@ private void validateSecondaryIndex(HoodieSparkEngineContext engineContext, Hood HoodieTableMetaClient metaClient, HoodieIndexDefinition indexDefinition) { String basePath = metaClient.getBasePath().toString(); String latestCompletedCommit = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline() - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().requestedTime(); JavaRDD secondaryKeys = readSecondaryKeys(engineContext, indexDefinition.getSourceFields(), basePath, latestCompletedCommit); secondaryKeys.persist(StorageLevel.MEMORY_AND_DISK()); @@ -1154,7 +1157,7 @@ private void validateRecordIndexCount(HoodieSparkEngineContext sparkEngineContex HoodieTableMetaClient metaClient) { String basePath = metaClient.getBasePath().toString(); String latestCompletedCommit = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline() - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().requestedTime(); long countKeyFromTable = sparkEngineContext.getSqlContext().read().format("hudi") .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT().key(),latestCompletedCommit) .load(basePath) @@ -1181,7 +1184,7 @@ private void validateRecordIndexContent(HoodieSparkEngineContext sparkEngineCont HoodieTableMetaClient metaClient) { String basePath = metaClient.getBasePath().toString(); String latestCompletedCommit = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline() - .filterCompletedInstants().lastInstant().get().getTimestamp(); + .filterCompletedInstants().lastInstant().get().requestedTime(); JavaPairRDD> keyToLocationOnFsRdd = getRecordLocationsFromFSBasedListing(sparkEngineContext, basePath, latestCompletedCommit); @@ -1480,11 +1483,10 @@ private boolean hasCommittedLogFiles( // The instant is completed, in active timeline // Checking commit metadata only as log files can only be written by COMMIT or DELTA_COMMIT if (!committedFilesMap.containsKey(instantTime)) { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes( - completedInstantsTimeline.getInstantDetails( - completedInstantsTimeline.filter(i -> i.getTimestamp().equals(instantTime)) - .firstInstant().get() - ).get(), + HoodieInstant instant = completedInstantsTimeline.filter(i -> i.requestedTime().equals(instantTime)) + .firstInstant().get(); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + instant, completedInstantsTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class ); committedFilesMap.put( diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java index ca3f7ab47b23..d625e7dafe82 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotCopier.java @@ -27,7 +27,6 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.util.Option; @@ -59,6 +58,8 @@ import scala.Tuple2; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.utilities.UtilHelpers.buildSparkConf; /** @@ -99,7 +100,7 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi LOG.warn("No commits present. Nothing to snapshot"); return; } - final String latestCommitTimestamp = latestCommit.get().getTimestamp(); + final String latestCommitTimestamp = latestCommit.get().requestedTime(); LOG.info(String.format("Starting to snapshot latest version files which are also no-late-than %s.", latestCommitTimestamp)); @@ -159,8 +160,8 @@ public void snapshot(JavaSparkContext jsc, String baseDir, final String outputDi if (fileStatus.isDirectory()) { return false; } - String instantTime = FSUtils.getCommitFromCommitFile(path.getName()); - return HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCommitTimestamp); + String instantTime = tableMetadata.getInstantFileNameParser().extractTimestamp(path.getName()); + return compareTimestamps(instantTime, LESSER_THAN_OR_EQUALS, latestCommitTimestamp); } }).toArray(FileStatus[]::new); for (FileStatus commitStatus : commitFilesToCopy) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java index 2faccb03192f..7a755aad136b 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HoodieSnapshotExporter.java @@ -28,7 +28,6 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.table.view.TableFileSystemView.BaseFileOnlyView; import org.apache.hudi.common.util.CollectionUtils; @@ -76,6 +75,8 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.utilities.UtilHelpers.buildSparkConf; /** @@ -150,7 +151,10 @@ public void export(JavaSparkContext jsc, Config cfg) throws IOException { } FileSystem sourceFs = HadoopFSUtils.getFs(cfg.sourceBasePath, jsc.hadoopConfiguration()); - final String latestCommitTimestamp = getLatestCommitTimestamp(sourceFs, cfg) + final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder() + .setConf(HadoopFSUtils.getStorageConfWithCopy(sourceFs.getConf())) + .setBasePath(cfg.sourceBasePath).build(); + final String latestCommitTimestamp = getLatestCommitTimestamp(tableMetadata) .orElseThrow(() -> { throw new HoodieSnapshotExporterException("No commits present. Nothing to snapshot."); }); @@ -165,20 +169,17 @@ public void export(JavaSparkContext jsc, Config cfg) throws IOException { LOG.info(String.format("The job needs to export %d partitions.", partitions.size())); if (cfg.outputFormat.equals(OutputFormatValidator.HUDI)) { - exportAsHudi(jsc, sourceFs, cfg, partitions, latestCommitTimestamp); + exportAsHudi(jsc, sourceFs, cfg, partitions, latestCommitTimestamp, tableMetadata); } else { exportAsNonHudi(jsc, sourceFs, cfg, partitions, latestCommitTimestamp); } createSuccessTag(outputFs, cfg); } - private Option getLatestCommitTimestamp(FileSystem fs, Config cfg) { - final HoodieTableMetaClient tableMetadata = HoodieTableMetaClient.builder() - .setConf(HadoopFSUtils.getStorageConfWithCopy(fs.getConf())) - .setBasePath(cfg.sourceBasePath).build(); + private Option getLatestCommitTimestamp(HoodieTableMetaClient tableMetadata) { Option latestCommit = tableMetadata.getActiveTimeline().getWriteTimeline() .filterCompletedInstants().lastInstant(); - return latestCommit.isPresent() ? Option.of(latestCommit.get().getTimestamp()) : Option.empty(); + return latestCommit.isPresent() ? Option.of(latestCommit.get().requestedTime()) : Option.empty(); } private List getPartitions(HoodieEngineContext engineContext, Config cfg, @@ -236,7 +237,8 @@ private void exportAsNonHudi(JavaSparkContext jsc, FileSystem sourceFs, } private void exportAsHudi(JavaSparkContext jsc, FileSystem sourceFs, - Config cfg, List partitions, String latestCommitTimestamp) throws IOException { + Config cfg, List partitions, String latestCommitTimestamp, + HoodieTableMetaClient metaClient) throws IOException { final int parallelism = cfg.parallelism == 0 ? jsc.defaultParallelism() : cfg.parallelism; final BaseFileOnlyView fsView = getBaseFileOnlyView(sourceFs, cfg); final HoodieEngineContext context = new HoodieSparkEngineContext(jsc); @@ -290,8 +292,8 @@ private void exportAsHudi(JavaSparkContext jsc, FileSystem sourceFs, if (fileStatus.isDirectory()) { return false; } - String instantTime = FSUtils.getCommitFromCommitFile(path.getName()); - return HoodieTimeline.compareTimestamps(instantTime, HoodieTimeline.LESSER_THAN_OR_EQUALS, latestCommitTimestamp); + String instantTime = metaClient.getInstantFileNameParser().extractTimestamp(path.getName()); + return compareTimestamps(instantTime, LESSER_THAN_OR_EQUALS, latestCommitTimestamp); } }).toArray(FileStatus[]::new); context.foreach(Arrays.asList(commitFilesToCopy), commitFile -> { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java index 5e50d851ca7a..9c867f804cad 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/checkpointing/InitialCheckpointFromAnotherHoodieTimelineProvider.java @@ -56,8 +56,8 @@ public String getCheckpoint() throws HoodieException { return anotherDsHoodieMetaClient.getCommitsTimeline().filterCompletedInstants().getReverseOrderedInstants() .map(instant -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(anotherDsHoodieMetaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieCommitMetadata commitMetadata = anotherDsHoodieMetaClient.getCommitMetadataSerDe() + .deserialize(instant, anotherDsHoodieMetaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); return commitMetadata.getMetadata(CHECKPOINT_KEY); } catch (IOException e) { diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java index ecad4c1875c5..81c479d706de 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/sources/HoodieIncrSource.java @@ -26,7 +26,7 @@ import org.apache.hudi.common.table.read.IncrementalQueryAnalyzer; import org.apache.hudi.common.table.read.IncrementalQueryAnalyzer.QueryContext; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -60,6 +60,8 @@ import static org.apache.hudi.DataSourceReadOptions.QUERY_TYPE; import static org.apache.hudi.DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL; import static org.apache.hudi.DataSourceReadOptions.START_COMMIT; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN_OR_EQUALS; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ConfigUtils.checkRequiredConfigProperties; import static org.apache.hudi.common.util.ConfigUtils.containsConfigProperty; import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; @@ -230,9 +232,9 @@ public Pair>, String> fetchNextBatch(Option lastCkpt endCompletionTime = newCheckpointAndPredicate.get().endCompletionTime; predicate = Option.of(newCheckpointAndPredicate.get().predicateFilter); instantTimeList = queryContext.getInstants().stream() - .filter(instant -> HoodieTimeline.compareTimestamps( - instant.getCompletionTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, newCheckpointAndPredicate.get().endCompletionTime)) - .map(HoodieInstant::getTimestamp) + .filter(instant -> compareTimestamps( + instant.getCompletionTime(), LESSER_THAN_OR_EQUALS, newCheckpointAndPredicate.get().endCompletionTime)) + .map(HoodieInstant::requestedTime) .collect(Collectors.toList()); } else { endCompletionTime = queryContext.getMaxCompletionTime(); @@ -246,8 +248,9 @@ public Pair>, String> fetchNextBatch(Option lastCkpt String.join("','", instantTimeList))); } else { // normal incremental query + TimelineLayout layout = TimelineLayout.fromVersion(queryContext.getActiveTimeline().getTimelineLayoutVersion()); String inclusiveStartCompletionTime = queryContext.getInstants().stream() - .min(HoodieInstant.COMPLETION_TIME_COMPARATOR) + .min(layout.getInstantComparator().completionTimeOrderedComparator()) .map(HoodieInstant::getCompletionTime) .get(); 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 a7faeef72df0..91116d52a14d 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 @@ -110,7 +110,7 @@ public static QueryInfo generateQueryInfo(JavaSparkContext jssc, String srcBaseP HoodieTimeline completedCommitTimeline = srcMetaClient.getCommitsAndCompactionTimeline().filterCompletedInstants(); final HoodieTimeline activeCommitTimeline = handleHollowCommitIfNeeded(completedCommitTimeline, srcMetaClient, handlingMode); Function timestampForLastInstant = instant -> handlingMode == HollowCommitHandling.USE_TRANSITION_TIME - ? instant.getCompletionTime() : instant.getTimestamp(); + ? instant.getCompletionTime() : instant.requestedTime(); String beginInstantTime = beginInstant.orElseGet(() -> { if (missingCheckpointStrategy != null) { if (missingCheckpointStrategy == MissingCheckpointStrategy.READ_LATEST) { @@ -134,11 +134,11 @@ public static QueryInfo generateQueryInfo(JavaSparkContext jssc, String srcBaseP if (!beginInstantTime.equals(DEFAULT_START_TIMESTAMP)) { Option previousInstant = activeCommitTimeline.findInstantBefore(beginInstantTime); if (previousInstant.isPresent()) { - previousInstantTime = previousInstant.get().getTimestamp(); + previousInstantTime = previousInstant.get().requestedTime(); } else { // if begin instant time matches first entry in active timeline, we can set previous = beginInstantTime - 1 if (activeCommitTimeline.filterCompletedInstants().firstInstant().isPresent() - && activeCommitTimeline.filterCompletedInstants().firstInstant().get().getTimestamp().equals(beginInstantTime)) { + && activeCommitTimeline.filterCompletedInstants().firstInstant().get().requestedTime().equals(beginInstantTime)) { previousInstantTime = String.valueOf(Long.parseLong(beginInstantTime) - 1); } } @@ -157,13 +157,13 @@ public static QueryInfo generateQueryInfo(JavaSparkContext jssc, String srcBaseP .findInstantsAfter(beginInstantTime, numInstantsPerFetch).getInstantsAsStream().reduce((x, y) -> y)); } return new QueryInfo(DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL(), previousInstantTime, - beginInstantTime, nthInstant.map(HoodieInstant::getTimestamp).orElse(beginInstantTime), + beginInstantTime, nthInstant.map(HoodieInstant::requestedTime).orElse(beginInstantTime), orderColumn, keyColumn, limitColumn); } else { // when MissingCheckpointStrategy is set to read everything until latest, trigger snapshot query. Option lastInstant = activeCommitTimeline.lastInstant(); return new QueryInfo(DataSourceReadOptions.QUERY_TYPE_SNAPSHOT_OPT_VAL(), - previousInstantTime, beginInstantTime, lastInstant.get().getTimestamp(), + previousInstantTime, beginInstantTime, lastInstant.get().requestedTime(), orderColumn, keyColumn, limitColumn); } } diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java index 54dcf673425d..a99aa372d87c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/HoodieStreamer.java @@ -40,8 +40,6 @@ import org.apache.hudi.common.table.HoodieTableConfig; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.table.timeline.HoodieInstant.State; -import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; @@ -822,8 +820,7 @@ protected Pair startService() { Option, JavaRDD>> scheduledCompactionInstantAndRDD = Option.ofNullable(streamSync.syncOnce()); if (scheduledCompactionInstantAndRDD.isPresent() && scheduledCompactionInstantAndRDD.get().getLeft().isPresent()) { LOG.info("Enqueuing new pending compaction instant (" + scheduledCompactionInstantAndRDD.get().getLeft() + ")"); - asyncCompactService.get().enqueuePendingAsyncServiceInstant(new HoodieInstant(State.REQUESTED, - HoodieTimeline.COMPACTION_ACTION, scheduledCompactionInstantAndRDD.get().getLeft().get())); + asyncCompactService.get().enqueuePendingAsyncServiceInstant(scheduledCompactionInstantAndRDD.get().getLeft().get()); asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions); if (asyncCompactService.get().hasError()) { error = true; @@ -834,7 +831,7 @@ protected Pair startService() { Option clusteringInstant = streamSync.getClusteringInstantOpt(); if (clusteringInstant.isPresent()) { LOG.info("Scheduled async clustering for instant: " + clusteringInstant.get()); - asyncClusteringService.get().enqueuePendingAsyncServiceInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.CLUSTERING_ACTION, clusteringInstant.get())); + asyncClusteringService.get().enqueuePendingAsyncServiceInstant(clusteringInstant.get()); asyncClusteringService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingClustering); if (asyncClusteringService.get().hasError()) { error = true; @@ -934,7 +931,7 @@ protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) { .setConf(HadoopFSUtils.getStorageConfWithCopy(hoodieSparkContext.hadoopConfiguration())) .setBasePath(cfg.targetBasePath).setLoadActiveTimelineOnLoad(true).build(); List pending = CompactionUtils.getPendingCompactionInstantTimes(meta); - pending.forEach(hoodieInstant -> asyncCompactService.get().enqueuePendingAsyncServiceInstant(hoodieInstant)); + pending.forEach(hoodieInstant -> asyncCompactService.get().enqueuePendingAsyncServiceInstant(hoodieInstant.requestedTime())); asyncCompactService.get().start(error -> true); try { asyncCompactService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingCompactions); @@ -958,7 +955,7 @@ protected Boolean onInitializingWriteClient(SparkRDDWriteClient writeClient) { .setLoadActiveTimelineOnLoad(true).build(); List pending = ClusteringUtils.getPendingClusteringInstantTimes(meta); LOG.info(format("Found %d pending clustering instants ", pending.size())); - pending.forEach(hoodieInstant -> asyncClusteringService.get().enqueuePendingAsyncServiceInstant(hoodieInstant)); + pending.forEach(hoodieInstant -> asyncClusteringService.get().enqueuePendingAsyncServiceInstant(hoodieInstant.requestedTime())); asyncClusteringService.get().start(error -> true); try { asyncClusteringService.get().waitTillPendingAsyncServiceInstantsReducesTo(cfg.maxPendingClustering); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java index 6bd48474c846..d5f12d17e99a 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/SparkSampleWritesUtils.java @@ -151,8 +151,8 @@ private static long getAvgSizeFromSampleWrites(JavaSparkContext jsc, String samp Option lastInstantOpt = metaClient.getCommitTimeline().filterCompletedInstants().lastInstant(); checkState(lastInstantOpt.isPresent(), "The only completed instant should be present in sample_writes table."); HoodieInstant instant = lastInstantOpt.get(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getCommitTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instant, metaClient.getCommitTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); long totalBytesWritten = commitMetadata.fetchTotalBytesWritten(); long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten(); return (long) Math.ceil((1.0 * totalBytesWritten) / totalRecordsWritten); diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java index 89640433dcd0..556e27c546a1 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamSync.java @@ -50,6 +50,7 @@ import org.apache.hudi.common.table.log.block.HoodieLogBlock; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.ConfigUtils; import org.apache.hudi.common.util.Option; @@ -141,6 +142,8 @@ import static org.apache.hudi.common.table.HoodieTableConfig.ARCHIVELOG_FOLDER; import static org.apache.hudi.common.table.HoodieTableConfig.HIVE_STYLE_PARTITIONING_ENABLE; import static org.apache.hudi.common.table.HoodieTableConfig.URL_ENCODE_PARTITIONING; +import static org.apache.hudi.common.table.timeline.InstantComparison.LESSER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; import static org.apache.hudi.common.util.ConfigUtils.getBooleanWithAltKeys; import static org.apache.hudi.common.util.ConfigUtils.removeConfigFromProps; import static org.apache.hudi.config.HoodieClusteringConfig.ASYNC_CLUSTERING_ENABLE; @@ -527,7 +530,7 @@ public Pair, JavaRDD> syncOnce() throws IOException private Option getLastPendingClusteringInstant(Option commitTimelineOpt) { if (commitTimelineOpt.isPresent()) { Option pendingClusteringInstant = commitTimelineOpt.get().getLastPendingClusterInstant(); - return pendingClusteringInstant.isPresent() ? Option.of(pendingClusteringInstant.get().getTimestamp()) : Option.empty(); + return pendingClusteringInstant.isPresent() ? Option.of(pendingClusteringInstant.get().requestedTime()) : Option.empty(); } return Option.empty(); } @@ -535,7 +538,7 @@ private Option getLastPendingClusteringInstant(Option co private Option getLastPendingCompactionInstant(Option commitTimelineOpt) { if (commitTimelineOpt.isPresent()) { Option pendingCompactionInstant = commitTimelineOpt.get().filterPendingCompactionTimeline().lastInstant(); - return pendingCompactionInstant.isPresent() ? Option.of(pendingCompactionInstant.get().getTimestamp()) : Option.empty(); + return pendingCompactionInstant.isPresent() ? Option.of(pendingCompactionInstant.get().requestedTime()) : Option.empty(); } return Option.empty(); } @@ -814,8 +817,8 @@ Option getCheckpointToResume(Option commitsTimelineOpt) //if previous checkpoint is an empty string, skip resume use Option.empty() String value = commitMetadata.getMetadata(CHECKPOINT_KEY); resumeCheckpointStr = Option.of(value); - } else if (HoodieTimeline.compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, - HoodieTimeline.LESSER_THAN, lastCommit.get().getTimestamp())) { + } else if (compareTimestamps(HoodieTimeline.FULL_BOOTSTRAP_INSTANT_TS, + LESSER_THAN, lastCommit.get().requestedTime())) { throw new HoodieStreamerException( "Unable to find previous checkpoint. Please double check if this table " + "was indeed built via delta streamer. Last Commit :" + lastCommit + ", Instants :" @@ -835,8 +838,9 @@ Option getCheckpointToResume(Option commitsTimelineOpt) protected Option> getLatestInstantAndCommitMetadataWithValidCheckpointInfo(HoodieTimeline timeline) throws IOException { return (Option>) timeline.getReverseOrderedInstants().map(instant -> { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() + .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_KEY)) || !StringUtils.isNullOrEmpty(commitMetadata.getMetadata(CHECKPOINT_RESET_KEY))) { return Option.of(Pair.of(instant.toString(), commitMetadata)); } else { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java index e2a7840ad1e9..11f6036fc223 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieIndexer.java @@ -207,7 +207,7 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata( context(), metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString()); HoodieTableMetaClient metadataMetaClient = metadata.getMetadataMetaClient(); - String mdtCommitTime = indexingInstant.getTimestamp(); + String mdtCommitTime = indexingInstant.requestedTime(); assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtCommitTime)); // Reverts both instants to inflight state, to simulate inflight indexing instants @@ -215,7 +215,7 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { metaClient = reload(metaClient); HoodieInstant mdtIndexingCommit = metadataMetaClient.getActiveTimeline() - .filter(i -> i.getTimestamp().equals(mdtCommitTime)) + .filter(i -> i.requestedTime().equals(mdtCommitTime)) .getInstants().get(0); metadataMetaClient.getActiveTimeline().revertToInflight(mdtIndexingCommit); metadataMetaClient = reload(metadataMetaClient); @@ -231,7 +231,7 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { metaClient = reload(metaClient); metadataMetaClient = reload(metadataMetaClient); // The delta commit from async indexer in metadata table should not be rolled back - assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.getTimestamp())); + assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.requestedTime())); assertTrue(metadataMetaClient.getActiveTimeline().getRollbackTimeline().empty()); // Simulate heartbeat timeout @@ -240,7 +240,7 @@ public void testIndexerWithWriterFinishingFirst() throws IOException { metaClient = reload(metaClient); metadataMetaClient = reload(metadataMetaClient); // The delta commit from async indexer in metadata table should be rolled back now - assertFalse(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.getTimestamp())); + assertFalse(metadataMetaClient.getActiveTimeline().containsInstant(mdtIndexingCommit.requestedTime())); assertEquals(1, metadataMetaClient.getActiveTimeline().getRollbackTimeline().countInstants()); HoodieInstant rollbackInstant = metadataMetaClient.getActiveTimeline() .getRollbackTimeline().firstInstant().get(); @@ -266,14 +266,14 @@ public void testIndexerWithWriterFinishingLast() throws IOException { // Transition the last commit to inflight HoodieInstant commit = metaClient.getActiveTimeline().lastInstant().get(); - String commitTime = commit.getTimestamp(); + String commitTime = commit.requestedTime(); metaClient.getActiveTimeline().revertToInflight(commit); HoodieBackedTableMetadata metadata = new HoodieBackedTableMetadata( context(), metaClient.getStorage(), metadataConfig, metaClient.getBasePath().toString()); HoodieTableMetaClient metadataMetaClient = metadata.getMetadataMetaClient(); HoodieInstant mdtCommit = metadataMetaClient.getActiveTimeline() - .filter(i -> i.getTimestamp().equals(commitTime)) + .filter(i -> i.requestedTime().equals(commitTime)) .getInstants().get(0); metadataMetaClient.getActiveTimeline().revertToInflight(mdtCommit); @@ -305,10 +305,10 @@ public void testIndexerWithWriterFinishingLast() throws IOException { assertTrue(metaClient.getActiveTimeline().containsInstant(commitTime)); assertTrue(metadataMetaClient.getActiveTimeline().containsInstant(commitTime)); assertTrue(metaClient.getActiveTimeline() - .filter(i -> i.getTimestamp().equals(commitTime)) + .filter(i -> i.requestedTime().equals(commitTime)) .getInstants().get(0).isInflight()); assertTrue(metadataMetaClient.getActiveTimeline() - .filter(i -> i.getTimestamp().equals(commitTime)) + .filter(i -> i.requestedTime().equals(commitTime)) .getInstants().get(0).isInflight()); assertTrue(metaClient.getActiveTimeline().getRollbackTimeline().empty()); assertTrue(metadataMetaClient.getActiveTimeline().getRollbackTimeline().empty()); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java index 488207adf8cd..cfd40f90a500 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/TestHoodieMetadataTableValidator.java @@ -34,6 +34,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimeGenerator; import org.apache.hudi.common.table.timeline.TimeGenerators; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageType; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.SerializationUtils; @@ -82,6 +83,9 @@ import static org.apache.hadoop.fs.FileUtil.copy; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.serializeCommitMetadata; +import static org.apache.hudi.common.testutils.HoodieTestUtils.COMMIT_METADATA_SER_DE; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.testutils.RawTripTestPayload.recordToString; import static org.apache.spark.sql.types.DataTypes.IntegerType; import static org.apache.spark.sql.types.DataTypes.StringType; @@ -352,7 +356,7 @@ public void testGetFSSecondaryKeyToRecordKeys() throws IOException { // There is one to one mapping between record key and secondary key String recKey = "row" + i++; Set recKeys = validator.getFSSecondaryKeyToRecordKeys(new HoodieSparkEngineContext(jsc, sqlContext), basePath, - metaClient.getActiveTimeline().lastInstant().get().getTimestamp(), "not_record_key_col", Collections.singletonList(secKey)) + metaClient.getActiveTimeline().lastInstant().get().requestedTime(), "not_record_key_col", Collections.singletonList(secKey)) .get(secKey); assertEquals(Collections.singleton(recKey), recKeys); } @@ -518,11 +522,11 @@ public void testAdditionalPartitionsinMDT(boolean testFailureCase) throws Interr if (testFailureCase) { // 3rd partition which is additional in MDT should have creation time before last instant in timeline. - String partition3CreationTime = HoodieActiveTimeline.createNewInstantTime(true, timeGenerator); + String partition3CreationTime = TimelineUtils.generateInstantTime(true, timeGenerator); Thread.sleep(100); - String lastIntantCreationTime = HoodieActiveTimeline.createNewInstantTime(true, timeGenerator); + String lastIntantCreationTime = TimelineUtils.generateInstantTime(true, timeGenerator); - HoodieInstant lastInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, lastIntantCreationTime); + HoodieInstant lastInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, lastIntantCreationTime); when(completedTimeline.lastInstant()).thenReturn(Option.of(lastInstant)); validator.setPartitionCreationTime(Option.of(partition3CreationTime)); // validate that exception is thrown since MDT has one additional partition. @@ -531,10 +535,11 @@ public void testAdditionalPartitionsinMDT(boolean testFailureCase) throws Interr }); } else { // 3rd partition creation time is > last completed instant - HoodieInstant lastInstant = new HoodieInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, HoodieActiveTimeline.createNewInstantTime(true, timeGenerator)); + HoodieInstant lastInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.COMMIT_ACTION, + TimelineUtils.generateInstantTime(true, timeGenerator)); when(completedTimeline.lastInstant()).thenReturn(Option.of(lastInstant)); Thread.sleep(100); - validator.setPartitionCreationTime(Option.of(HoodieActiveTimeline.createNewInstantTime(true, timeGenerator))); + validator.setPartitionCreationTime(Option.of(TimelineUtils.generateInstantTime(true, timeGenerator))); // validate that all 3 partitions are returned assertEquals(mdtPartitions, validator.validatePartitions(engineContext, baseStoragePath, metaClient)); @@ -566,7 +571,7 @@ public void testAdditionalFilesinMetadata(Integer lastNFileSlices) throws Except // let's add a log file entry to the commit history and filesystem by directly modifying the commit so FS based listing and MDT based listing diverges. HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieInstant instantToOverwrite = timeline.getInstants().get(1); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instantToOverwrite).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = COMMIT_METADATA_SER_DE.deserialize(instantToOverwrite, timeline.getInstantDetails(instantToOverwrite).get(), HoodieCommitMetadata.class); HoodieWriteStat writeStatToCopy = commitMetadata.getPartitionToWriteStats().entrySet().stream().flatMap(entry -> entry.getValue().stream()) .filter(writeStat -> FSUtils.isLogFile(writeStat.getPath())).findFirst().get(); String newLogFilePath = writeStatToCopy.getPath() + "1"; @@ -576,9 +581,9 @@ public void testAdditionalFilesinMetadata(Integer lastNFileSlices) throws Except FileSystem fs = HadoopFSUtils.getFs(newLogFilePath, new Configuration(false)); fs.copyFromLocalFile(new Path(basePath, writeStatToCopy.getPath()), new Path(basePath, newLogFilePath)); // remove the existing instant and rewrite with the new metadata - assertTrue(fs.delete(new Path(basePath, String.format(".hoodie/%s", instantToOverwrite.getFileName())))); - timeline.saveAsComplete(new HoodieInstant(HoodieInstant.State.INFLIGHT, instantToOverwrite.getAction(), instantToOverwrite.getTimestamp(), instantToOverwrite.getCompletionTime()), - serializeCommitMetadata(commitMetadata)); + assertTrue(fs.delete(new Path(basePath, String.format(".hoodie/%s", INSTANT_FILE_NAME_GENERATOR.getFileName(instantToOverwrite))))); + timeline.saveAsComplete(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instantToOverwrite.getAction(), instantToOverwrite.requestedTime(), + instantToOverwrite.getCompletionTime()), serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata)); for (int i = 0; i < 5; i++) { inserts.write().format("hudi").options(writeOptions) @@ -700,9 +705,9 @@ public void testRliValidationFalsePositiveCase() throws IOException { HoodieTableMetaClient metaClient = HoodieTableMetaClient.builder().setBasePath(basePath).setConf(HadoopFSUtils.getStorageConfWithCopy(jsc.hadoopConfiguration())).build(); // moving out the completed commit meta file to a temp location HoodieInstant lastInstant = metaClient.getActiveTimeline().filterCompletedInstants().lastInstant().get(); - String latestCompletedCommitMetaFile = basePath + "/.hoodie/" + lastInstant.getFileName(); + String latestCompletedCommitMetaFile = basePath + "/.hoodie/" + INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant); String tempDir = getTempLocation(); - String destFilePath = tempDir + "/" + lastInstant.getFileName(); + String destFilePath = tempDir + "/" + INSTANT_FILE_NAME_GENERATOR.getFileName(lastInstant); FileUtil.move(latestCompletedCommitMetaFile, destFilePath); MockHoodieMetadataTableValidatorForRli validator = new MockHoodieMetadataTableValidatorForRli(jsc, config); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java index d125dc335c71..6579b61723b8 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/HoodieDeltaStreamerTestBase.java @@ -79,6 +79,9 @@ import java.util.concurrent.TimeUnit; import java.util.function.Function; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.table.timeline.InstantComparison.compareTimestamps; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.createMetaClient; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -490,16 +493,16 @@ static void addCommitToTimeline(HoodieTableMetaClient metaClient, WriteOperation commitMetadata.setOperationType(writeOperationType); extraMetadata.forEach((k, v) -> commitMetadata.getExtraMetadata().put(k, v)); String commitTime = metaClient.createNewInstantTime(); - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(HoodieInstant.State.REQUESTED, commitActiontype, commitTime)); - HoodieInstant inflightInstant = new HoodieInstant(HoodieInstant.State.INFLIGHT, commitActiontype, commitTime); + metaClient.getActiveTimeline().createNewInstant(INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.REQUESTED, commitActiontype, commitTime)); + HoodieInstant inflightInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, commitActiontype, commitTime); metaClient.getActiveTimeline().createNewInstant(inflightInstant); if (commitActiontype.equals(HoodieTimeline.CLUSTERING_ACTION)) { metaClient.getActiveTimeline().transitionClusterInflightToComplete(true, inflightInstant, - TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); } else { metaClient.getActiveTimeline().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, commitActiontype, commitTime), - TimelineMetadataUtils.serializeCommitMetadata(commitMetadata)); + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, commitActiontype, commitTime), + TimelineMetadataUtils.serializeCommitMetadata(metaClient.getCommitMetadataSerDe(), commitMetadata)); } } @@ -686,7 +689,7 @@ static HoodieInstant assertCommitMetadata(String expected, String tablePath, int HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieInstant lastInstant = timeline.lastInstant().get(); HoodieCommitMetadata commitMetadata = - HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); + meta.getCommitMetadataSerDe().deserialize(lastInstant, timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); assertEquals(totalCommits, timeline.countInstants()); assertEquals(expected, commitMetadata.getMetadata(HoodieStreamer.CHECKPOINT_KEY)); return lastInstant; @@ -766,7 +769,7 @@ static void assertAtLeastNCommitsAfterRollback(int minExpectedRollback, int minE HoodieInstant firstRollback = timeline.getInstants().get(0); // HoodieTimeline commitsTimeline = meta.getActiveTimeline().filterCompletedInstants() - .filter(instant -> HoodieTimeline.compareTimestamps(instant.getTimestamp(), HoodieTimeline.GREATER_THAN, firstRollback.getTimestamp())); + .filter(instant -> compareTimestamps(instant.requestedTime(), GREATER_THAN, firstRollback.requestedTime())); int numCommits = commitsTimeline.countInstants(); assertTrue(minExpectedCommits <= numCommits, "Got=" + numCommits + ", exp >=" + minExpectedCommits); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java index 536cd21c6714..716aae780529 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamer.java @@ -51,7 +51,9 @@ import org.apache.hudi.common.table.TableSchemaResolver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.InstantComparison; import org.apache.hudi.common.table.timeline.TimelineUtils; +import org.apache.hudi.common.table.timeline.versioning.DefaultCommitMetadataSerDe; import org.apache.hudi.common.table.view.FileSystemViewManager; import org.apache.hudi.common.table.view.HoodieTableFileSystemView; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -165,6 +167,8 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.InstantComparison.GREATER_THAN; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.testutils.HoodieClientTestUtils.createMetaClient; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; @@ -810,8 +814,8 @@ public void testDeltaSyncWithPendingClustering() throws Exception { cfg.retryLastPendingInlineClusteringJob = true; HoodieDeltaStreamer ds2 = new HoodieDeltaStreamer(cfg, jsc); ds2.sync(); - String completeClusteringTimeStamp = meta.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant().get().getTimestamp(); - assertEquals(clusteringRequest.getTimestamp(), completeClusteringTimeStamp); + String completeClusteringTimeStamp = meta.reloadActiveTimeline().getCompletedReplaceTimeline().lastInstant().get().requestedTime(); + assertEquals(clusteringRequest.requestedTime(), completeClusteringTimeStamp); TestHelpers.assertAtLeastNCommits(2, tableBasePath); TestHelpers.assertAtLeastNReplaceCommits(1, tableBasePath); } @@ -847,7 +851,7 @@ public void testDeltaSyncWithPendingCompaction() throws Exception { HoodieTableMetaClient meta = HoodieTestUtils.createMetaClient(storage, tableBasePath); HoodieTimeline timeline = meta.getActiveTimeline().getCommitAndReplaceTimeline().filterCompletedInstants(); HoodieInstant commitInstant = timeline.lastInstant().get(); - String commitFileName = tableBasePath + "/.hoodie/" + commitInstant.getFileName(); + String commitFileName = tableBasePath + "/.hoodie/" + INSTANT_FILE_NAME_GENERATOR.getFileName(commitInstant); fs.delete(new Path(commitFileName), false); // sync again @@ -1234,12 +1238,12 @@ public void testAsyncClusteringJobWithRetry(boolean retryLastFailedClusteringJob HoodieClusteringJob scheduleAndExecute = initialHoodieClusteringJob(tableBasePath, null, false, "scheduleAndExecute", retryLastFailedClusteringJob, HoodieRecordType.AVRO); scheduleAndExecute.cluster(0); - String completeClusteringTimeStamp = meta.getActiveTimeline().reload().getCompletedReplaceTimeline().lastInstant().get().getTimestamp(); + String completeClusteringTimeStamp = meta.getActiveTimeline().reload().getCompletedReplaceTimeline().lastInstant().get().requestedTime(); if (retryLastFailedClusteringJob) { - assertEquals(clusteringRequest.getTimestamp(), completeClusteringTimeStamp); + assertEquals(clusteringRequest.requestedTime(), completeClusteringTimeStamp); } else { - assertFalse(clusteringRequest.getTimestamp().equalsIgnoreCase(completeClusteringTimeStamp)); + assertFalse(clusteringRequest.requestedTime().equalsIgnoreCase(completeClusteringTimeStamp)); } UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); } @@ -1558,7 +1562,7 @@ public void testBulkInsertsAndUpsertsWithSQLBasedTransformerFor2StepPipeline() t assertTrue(hiveClient.tableExists(tableName), "Table " + tableName + " should exist"); assertEquals(3, hiveClient.getAllPartitions(tableName).size(), "Table partitions should match the number of partitions we wrote"); - assertEquals(lastInstantForUpstreamTable.getTimestamp(), + assertEquals(lastInstantForUpstreamTable.requestedTime(), hiveClient.getLastCommitTimeSynced(tableName).get(), "The last commit that was synced should be updated in the TBLPROPERTIES"); UtilitiesTestBase.Helpers.deleteFileFromDfs(fs, tableBasePath); @@ -1683,12 +1687,12 @@ public void testFilterDupes() throws Exception { ds2.sync(); mClient = createMetaClient(jsc, tableBasePath); HoodieInstant newLastFinished = mClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get(); - assertTrue(HoodieTimeline.compareTimestamps(newLastFinished.getTimestamp(), HoodieTimeline.GREATER_THAN, lastFinished.getTimestamp() + assertTrue(InstantComparison.compareTimestamps(newLastFinished.requestedTime(), GREATER_THAN, lastFinished.requestedTime() )); // Ensure it is empty - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(mClient.getActiveTimeline().getInstantDetails(newLastFinished).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = new DefaultCommitMetadataSerDe() + .deserialize(newLastFinished, mClient.getActiveTimeline().getInstantDetails(newLastFinished).get(), HoodieCommitMetadata.class); System.out.println("New Commit Metadata=" + commitMetadata); assertTrue(commitMetadata.getPartitionToWriteStats().isEmpty()); @@ -1794,8 +1798,8 @@ private void assertValidSchemaAndOperationTypeInCommitMetadata(HoodieInstant ins HoodieTableMetaClient metaClient, WriteOperationType operationType) { try { - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); assertFalse(StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY))); assertEquals(operationType, commitMetadata.getOperationType()); } catch (IOException ioException) { @@ -2047,21 +2051,21 @@ public void testDeltaStreamerMultiwriterCheckpoint() throws Exception { List instants = metaClient.getCommitsTimeline().getInstants(); ObjectMapper objectMapper = new ObjectMapper(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getCommitsTimeline().getInstantDetails(instants.get(0)).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instants.get(0), metaClient.getCommitsTimeline().getInstantDetails(instants.get(0)).get(), HoodieCommitMetadata.class); Map checkpointVals = objectMapper.readValue(commitMetadata.getExtraMetadata().get(HoodieDeltaStreamer.CHECKPOINT_KEY), Map.class); String parquetFirstcheckpoint = checkpointVals.get("parquet"); assertNotNull(parquetFirstcheckpoint); - commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getCommitsTimeline().getInstantDetails(instants.get(1)).get(), HoodieCommitMetadata.class); + commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instants.get(1), metaClient.getCommitsTimeline().getInstantDetails(instants.get(1)).get(), HoodieCommitMetadata.class); checkpointVals = objectMapper.readValue(commitMetadata.getExtraMetadata().get(HoodieDeltaStreamer.CHECKPOINT_KEY), Map.class); String kafkaCheckpoint = checkpointVals.get("kafka"); assertNotNull(kafkaCheckpoint); assertEquals(parquetFirstcheckpoint, checkpointVals.get("parquet")); - commitMetadata = HoodieCommitMetadata - .fromBytes(metaClient.getCommitsTimeline().getInstantDetails(instants.get(2)).get(), HoodieCommitMetadata.class); + commitMetadata = metaClient.getCommitMetadataSerDe() + .deserialize(instants.get(2), metaClient.getCommitsTimeline().getInstantDetails(instants.get(2)).get(), HoodieCommitMetadata.class); checkpointVals = objectMapper.readValue(commitMetadata.getExtraMetadata().get(HoodieDeltaStreamer.CHECKPOINT_KEY), Map.class); String parquetSecondCheckpoint = checkpointVals.get("parquet"); assertNotNull(parquetSecondCheckpoint); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerWithMultiWriter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerWithMultiWriter.java index 7fe0d2d311ed..d5a14cd4a4c6 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerWithMultiWriter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/deltastreamer/TestHoodieDeltaStreamerWithMultiWriter.java @@ -27,6 +27,7 @@ import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.config.HoodieCleanConfig; import org.apache.hudi.config.HoodieCompactionConfig; @@ -147,8 +148,9 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta cfgBackfillJob.continuousMode = false; HoodieTableMetaClient meta = createMetaClient(new HadoopStorageConfiguration(hadoopConf), tableBasePath); HoodieTimeline timeline = meta.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() + .deserialize(timeline.firstInstant().get(), timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob.configs.add(String.format("%s=%d", SourceTestConfig.MAX_UNIQUE_RECORDS_PROP.key(), totalRecords)); cfgBackfillJob.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); @@ -215,8 +217,8 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp cfgBackfillJob2.continuousMode = false; HoodieTableMetaClient meta = createMetaClient(new HadoopStorageConfiguration(hadoopConf), tableBasePath); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - HoodieCommitMetadata commitMetadata = HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = meta.getCommitMetadataSerDe().deserialize( + timeline.firstInstant().get(), timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); cfgBackfillJob2.checkpoint = commitMetadata.getMetadata(CHECKPOINT_KEY); cfgBackfillJob2.configs.add(String.format("%s=%d", SourceTestConfig.MAX_UNIQUE_RECORDS_PROP.key(), totalRecords)); cfgBackfillJob2.configs.add(String.format("%s=false", HoodieCleanConfig.AUTO_CLEAN.key())); @@ -330,9 +332,8 @@ private static HoodieCommitMetadata getLatestMetadata(HoodieTableMetaClient meta throws IOException { HoodieTimeline timeline = meta.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); - return HoodieCommitMetadata - .fromBytes(timeline.getInstantDetails(timeline.lastInstant().get()).get(), - HoodieCommitMetadata.class); + return meta.getCommitMetadataSerDe().deserialize( + timeline.firstInstant().get(), timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); } private static TypedProperties prepareMultiWriterProps(HoodieStorage storage, String basePath, @@ -394,7 +395,7 @@ private void runJobsInParallel(String tableBasePath, HoodieTableType tableType, ExecutorService service = Executors.newFixedThreadPool(2); HoodieTableMetaClient meta = createMetaClient(new HadoopStorageConfiguration(hadoopConf), tableBasePath); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - String lastSuccessfulCommit = timeline.lastInstant().get().getTimestamp(); + String lastSuccessfulCommit = timeline.lastInstant().get().requestedTime(); // Condition for parallel ingestion job Function conditionForRegularIngestion = (r) -> { if (tableType.equals(HoodieTableType.MERGE_ON_READ)) { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java index 0831fd6ca9ac..de5ea9493e14 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/functional/TestHDFSParquetImporter.java @@ -18,8 +18,8 @@ package org.apache.hudi.utilities.functional; -import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestUtils; import org.apache.hudi.storage.StoragePath; @@ -238,7 +238,7 @@ public void testImportWithUpsert() throws IOException, ParseException { public List createInsertRecords(Path srcFolder) throws ParseException, IOException { Path srcFile = new Path(srcFolder.toString(), "file1.parquet"); - long startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime() / 1000; + long startTime = TimelineUtils.parseDateFromInstantTime("20170203000000").getTime() / 1000; List records = new ArrayList(); for (long recordNum = 0; recordNum < 96; recordNum++) { records.add(new HoodieTestDataGenerator().generateGenericRecord(Long.toString(recordNum), "0", "rider-" + recordNum, "driver-" + recordNum, startTime + TimeUnit.HOURS.toSeconds(recordNum))); @@ -255,7 +255,7 @@ public List createInsertRecords(Path srcFolder) throws ParseExcep public List createUpsertRecords(Path srcFolder) throws ParseException, IOException { Path srcFile = new Path(srcFolder.toString(), "file1.parquet"); - long startTime = HoodieActiveTimeline.parseDateFromInstantTime("20170203000000").getTime() / 1000; + long startTime = TimelineUtils.parseDateFromInstantTime("20170203000000").getTime() / 1000; List records = new ArrayList(); // 10 for update HoodieTestDataGenerator dataGen = new HoodieTestDataGenerator(); diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java index 9f313f0926c5..2efb7c51c8f0 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/offlinejob/TestHoodieClusteringJob.java @@ -41,6 +41,7 @@ import java.util.Properties; import static org.apache.hudi.common.table.HoodieTableMetaClient.METAFOLDER_NAME; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA; import static org.apache.hudi.utilities.UtilHelpers.PURGE_PENDING_INSTANT; import static org.apache.hudi.utilities.testutils.UtilitiesTestBase.Helpers.deleteFileFromDfs; @@ -115,12 +116,12 @@ public void testPurgePendingInstants() throws Exception { // remove the completed instant from timeline and trigger purge of pending clustering instant. HoodieInstant latestClusteringInstant = metaClient.getActiveTimeline() .filterCompletedInstantsOrRewriteTimeline().getCompletedReplaceTimeline().getInstants().get(0); - String completedFilePath = tableBasePath + "/" + METAFOLDER_NAME + "/" + latestClusteringInstant.getFileName(); + String completedFilePath = tableBasePath + "/" + METAFOLDER_NAME + "/" + INSTANT_FILE_NAME_GENERATOR.getFileName(latestClusteringInstant); deleteFileFromDfs(fs, completedFilePath); // trigger purge. hoodieCluster = - getClusteringConfigForPurge(tableBasePath, true, PURGE_PENDING_INSTANT, latestClusteringInstant.getTimestamp()); + getClusteringConfigForPurge(tableBasePath, true, PURGE_PENDING_INSTANT, latestClusteringInstant.requestedTime()); hoodieCluster.cluster(0); // validate that there are no clustering commits in timeline. HoodieOfflineJobTestBase.TestHelpers.assertNClusteringCommits(0, tableBasePath); @@ -130,7 +131,7 @@ public void testPurgePendingInstants() throws Exception { for (int i = 0; i < fullPartitionPaths.length; i++) { fullPartitionPaths[i] = String.format("%s/%s/*", tableBasePath, dataGen.getPartitionPaths()[i]); } - assertEquals(0, HoodieClientTestUtils.read(jsc, tableBasePath, sqlContext, storage, fullPartitionPaths).filter("_hoodie_commit_time = " + latestClusteringInstant.getTimestamp()).count(), + assertEquals(0, HoodieClientTestUtils.read(jsc, tableBasePath, sqlContext, storage, fullPartitionPaths).filter("_hoodie_commit_time = " + latestClusteringInstant.requestedTime()).count(), "Must not contain any records w/ clustering instant time"); } diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java index 5ad086ebd235..34c63dd52155 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/TestHoodieIncrSource.java @@ -74,6 +74,7 @@ import static org.apache.hudi.common.model.WriteOperationType.INSERT; import static org.apache.hudi.common.model.WriteOperationType.UPSERT; import static org.apache.hudi.common.testutils.HoodieTestUtils.DEFAULT_PARTITION_PATHS; +import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_GENERATOR; import static org.apache.hudi.common.testutils.HoodieTestUtils.RAW_TRIPS_TEST_NAME; import static org.apache.hudi.testutils.Assertions.assertNoWriteErrors; import static org.apache.hudi.utilities.sources.helpers.IncrSourceHelper.MissingCheckpointStrategy.READ_UPTO_LATEST_COMMIT; @@ -218,7 +219,7 @@ public void testHoodieIncrSourceInflightCommitBeforeCompletedCommit(HoodieTableT // The checkpoint should not go past this commit HoodieActiveTimeline activeTimeline = metaClient.getActiveTimeline(); HoodieInstant instant4 = activeTimeline - .filter(instant -> instant.getTimestamp().equals(inserts.get(4).getInstantTime())).firstInstant().get(); + .filter(instant -> instant.requestedTime().equals(inserts.get(4).getInstantTime())).firstInstant().get(); Option instant4CommitData = activeTimeline.getInstantDetails(instant4); activeTimeline.revertToInflight(instant4); metaClient.reloadActiveTimeline(); @@ -266,12 +267,12 @@ public void testHoodieIncrSourceInflightCommitBeforeCompletedCommit(HoodieTableT inserts.get(5).getCompletionTime()); activeTimeline.reload().saveAsComplete( - new HoodieInstant(HoodieInstant.State.INFLIGHT, instant4.getAction(), inserts.get(4).getInstantTime()), + INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, instant4.getAction(), inserts.get(4).getInstantTime()), instant4CommitData); // find instant4's new completion time String instant4CompletionTime = activeTimeline.reload().getInstantsAsStream() - .filter(instant -> instant.getTimestamp().equals(instant4.getTimestamp())) + .filter(instant -> instant.requestedTime().equals(instant4.requestedTime())) .findFirst().get().getCompletionTime(); // After the inflight commit completes, the checkpoint should move on after incremental pull @@ -347,14 +348,14 @@ public void testHoodieIncrSourceWithPendingTableServices(HoodieTableType tableTy .filter(instant -> ClusteringUtils.getClusteringPlan(metaClient, instant).isPresent()) .firstInstant(); assertTrue(clusteringInstant.isPresent()); - assertTrue(clusteringInstant.get().getTimestamp().compareTo(latestCommitTimestamp) < 0); + assertTrue(clusteringInstant.get().requestedTime().compareTo(latestCommitTimestamp) < 0); if (tableType == MERGE_ON_READ) { // Pending compaction exists Option compactionInstant = metaClient.getActiveTimeline().filterPendingCompactionTimeline().firstInstant(); assertTrue(compactionInstant.isPresent()); - assertTrue(compactionInstant.get().getTimestamp().compareTo(latestCommitTimestamp) < 0); + assertTrue(compactionInstant.get().requestedTime().compareTo(latestCommitTimestamp) < 0); } // test SnapshotLoadQuerySplitter to split snapshot query . @@ -662,7 +663,7 @@ public HoodieInstant getInstant() { } public String getInstantTime() { - return instant.getTimestamp(); + return instant.requestedTime(); } public String getCompletionTime() { diff --git a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestSnapshotQuerySplitterImpl.java b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestSnapshotQuerySplitterImpl.java index b6bfe12244e2..afe147818be4 100644 --- a/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestSnapshotQuerySplitterImpl.java +++ b/hudi-utilities/src/test/java/org/apache/hudi/utilities/sources/helpers/TestSnapshotQuerySplitterImpl.java @@ -61,7 +61,7 @@ public Option getNextCheckpointWithPredicates(Dataset< int maxRowsPerBatch = properties.getInteger(MAX_ROWS_PER_BATCH, 1); List instantTimeList = queryContext.getInstantTimeList(); Map instantToCompletionTimeMap = queryContext.getInstants().stream() - .collect(Collectors.toMap(HoodieInstant::getTimestamp, HoodieInstant::getCompletionTime)); + .collect(Collectors.toMap(HoodieInstant::requestedTime, HoodieInstant::getCompletionTime)); Map> completionTimeToStats = df.select(col(COMMIT_TIME_METADATA_FIELD), col(PARTITION_PATH_METADATA_FIELD)) .filter(col(COMMIT_TIME_METADATA_FIELD).isin(instantTimeList.toArray()))