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 d5a00437a22da..417980f2e4cc8 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 @@ -78,7 +78,7 @@ public String showCleans( List rows = new ArrayList<>(); for (HoodieInstant clean : cleans) { HoodieCleanMetadata cleanMetadata = - TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(clean).get()); + TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantContentStream(clean)); rows.add(new Comparable[] {clean.requestedTime(), cleanMetadata.getEarliestCommitToRetain(), cleanMetadata.getTotalFilesDeleted(), cleanMetadata.getTimeTakenInMillis()}); } @@ -110,7 +110,7 @@ public String showCleanPartitions( } HoodieCleanMetadata cleanMetadata = - TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(cleanInstant).get()); + TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantContentStream(cleanInstant)); List rows = new ArrayList<>(); for (Map.Entry entry : cleanMetadata.getPartitionMetadata().entrySet()) { String path = entry.getKey(); 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 7a3d6fdcc9b20..5fd4ac32813a6 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,11 +25,11 @@ 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.CommitMetadataSerDe; 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.InstantComparator; -import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.NumericUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; @@ -72,19 +72,18 @@ private String printCommits(HoodieTimeline timeline, .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { - if (timeline.getInstantDetails(commit).isPresent()) { - final HoodieCommitMetadata commitMetadata = getCommitMetadata(HoodieCLI.getTableMetaClient().getActiveTimeline(), - commit, - HoodieCommitMetadata.class); - rows.add(new Comparable[] {commit.requestedTime(), - commitMetadata.fetchTotalBytesWritten(), - commitMetadata.fetchTotalFilesInsert(), - commitMetadata.fetchTotalFilesUpdated(), - commitMetadata.fetchTotalPartitionsWritten(), - commitMetadata.fetchTotalRecordsWritten(), - commitMetadata.fetchTotalUpdateRecordsWritten(), - commitMetadata.fetchTotalWriteErrors()}); - } + CommitMetadataSerDe serDe = HoodieCLI.getTableMetaClient().getCommitMetadataSerDe(); + final HoodieCommitMetadata commitMetadata = serDe.deserialize( + commit, timeline.getInstantContentStream(commit), + HoodieCommitMetadata.class); + rows.add(new Comparable[] {commit.requestedTime(), + commitMetadata.fetchTotalBytesWritten(), + commitMetadata.fetchTotalFilesInsert(), + commitMetadata.fetchTotalFilesUpdated(), + commitMetadata.fetchTotalPartitionsWritten(), + commitMetadata.fetchTotalRecordsWritten(), + commitMetadata.fetchTotalUpdateRecordsWritten(), + commitMetadata.fetchTotalWriteErrors()}); } final Map> fieldNameToConverterMap = new HashMap<>(); @@ -111,25 +110,23 @@ private String printCommitsWithMetadata(HoodieTimeline timeline, .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { - if (timeline.getInstantDetails(commit).isPresent()) { - final HoodieCommitMetadata commitMetadata = HoodieCLI.getTableMetaClient().getCommitMetadataSerDe().deserialize( - commit, - timeline.getInstantDetails(commit).get(), - HoodieCommitMetadata.class); - - for (Map.Entry> partitionWriteStat : - commitMetadata.getPartitionToWriteStats().entrySet()) { - for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { - if (StringUtils.isNullOrEmpty(partition) || partition.equals(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(), - hoodieWriteStat.getTotalLogBlocks(), hoodieWriteStat.getTotalCorruptLogBlock(), - hoodieWriteStat.getTotalRollbackBlocks(), hoodieWriteStat.getTotalLogRecords(), - hoodieWriteStat.getTotalUpdatedRecordsCompacted(), hoodieWriteStat.getTotalWriteBytes() - }); - } + final HoodieCommitMetadata commitMetadata = HoodieCLI.getTableMetaClient().getCommitMetadataSerDe().deserialize( + commit, + timeline.getInstantContentStream(commit), + HoodieCommitMetadata.class); + + for (Map.Entry> partitionWriteStat : + commitMetadata.getPartitionToWriteStats().entrySet()) { + for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { + if (StringUtils.isNullOrEmpty(partition) || partition.equals(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(), + hoodieWriteStat.getTotalLogBlocks(), hoodieWriteStat.getTotalCorruptLogBlock(), + hoodieWriteStat.getTotalRollbackBlocks(), hoodieWriteStat.getTotalLogRecords(), + hoodieWriteStat.getTotalUpdatedRecordsCompacted(), hoodieWriteStat.getTotalWriteBytes() + }); } } } 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 f98ea7cecbb85..d0f8719c86479 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 @@ -181,7 +181,7 @@ public String compactionShowArchived( try { archivedTimeline.loadCompactionDetailsInMemory(compactionInstantTime); HoodieCompactionPlan compactionPlan = - TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get()); + TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantContentStream(instant)); return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition); } finally { archivedTimeline.clearInstantDetailsFromMemory(compactionInstantTime); @@ -371,7 +371,7 @@ Function compactionPlanReader( private HoodieCompactionPlan readCompactionPlanForArchivedTimeline(HoodieArchivedTimeline archivedTimeline, HoodieInstant instant) { try { - return TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get()); + return TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantContentStream(instant)); } catch (Exception e) { throw new HoodieException(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 617a820f87fcd..24cd160c7a097 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 @@ -30,7 +30,6 @@ 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; import org.springframework.shell.standard.ShellComponent; import org.springframework.shell.standard.ShellMethod; @@ -118,14 +117,12 @@ private String printDiffWithMetadata(HoodieTimeline timeline, Integer limit, Str .getInstantsAsStream().sorted(instantComparator.requestedTimeOrderedComparator().reversed()).collect(Collectors.toList()); for (final HoodieInstant commit : commits) { - Option instantDetails = timeline.getInstantDetails(commit); - if (instantDetails.isPresent()) { - HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize(commit, instantDetails.get(), HoodieCommitMetadata.class); - for (Map.Entry> partitionWriteStat : - commitMetadata.getPartitionToWriteStats().entrySet()) { - for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { - populateRows(rows, commit, hoodieWriteStat, diffEntity, diffEntityChecker); - } + HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize( + commit, timeline.getInstantContentStream(commit), HoodieCommitMetadata.class); + for (Map.Entry> partitionWriteStat : + commitMetadata.getPartitionToWriteStats().entrySet()) { + for (HoodieWriteStat hoodieWriteStat : partitionWriteStat.getValue()) { + populateRows(rows, commit, hoodieWriteStat, diffEntity, diffEntityChecker); } } } 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 6f4cc3be910cf..f3d774e5f2328 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 @@ -28,7 +28,6 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; -import org.apache.hudi.common.util.Option; import org.springframework.shell.standard.ShellComponent; import org.springframework.shell.standard.ShellMethod; @@ -98,9 +97,8 @@ public String showRestore( private void addDetailsOfCompletedRestore(HoodieActiveTimeline activeTimeline, List rows, HoodieInstant restoreInstant) throws IOException { HoodieRestoreMetadata instantMetadata; - Option instantDetails = activeTimeline.getInstantDetails(restoreInstant); instantMetadata = TimelineMetadataUtils - .deserializeAvroMetadata(instantDetails.get(), HoodieRestoreMetadata.class); + .deserializeAvroMetadata(activeTimeline.getInstantContentStream(restoreInstant), HoodieRestoreMetadata.class); for (String rolledbackInstant : instantMetadata.getInstantsToRollback()) { Comparable[] row = createDataRow(instantMetadata.getStartRestoreTime(), rolledbackInstant, @@ -122,9 +120,8 @@ private void addDetailsOfInflightRestore(HoodieActiveTimeline activeTimeline, Li private HoodieRestorePlan getRestorePlan(HoodieActiveTimeline activeTimeline, HoodieInstant restoreInstant) throws IOException { 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); + .deserializeAvroMetadata(activeTimeline.getInstantContentStream(instantKey), HoodieRestorePlan.class); return restorePlan; } 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 03c40ea58f5cc..9f616bd640aa3 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 @@ -64,7 +64,7 @@ public String showRollbacks( rollback.getInstants().forEach(instant -> { try { HoodieRollbackMetadata metadata = TimelineMetadataUtils - .deserializeAvroMetadata(activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + .deserializeAvroMetadata(activeTimeline.getInstantContentStream(instant), HoodieRollbackMetadata.class); metadata.getCommitsRollback().forEach(c -> { Comparable[] row = new Comparable[5]; row[0] = metadata.getStartRollbackTime(); @@ -98,7 +98,7 @@ public String showRollback( HoodieActiveTimeline activeTimeline = HoodieCLI.getTableMetaClient().getActiveTimeline(); final List rows = new ArrayList<>(); HoodieRollbackMetadata metadata = TimelineMetadataUtils.deserializeAvroMetadata( - activeTimeline.getInstantDetails(HoodieCLI.getTableMetaClient().createNewInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)).get(), + activeTimeline.getInstantContentStream(HoodieCLI.getTableMetaClient().createNewInstant(State.COMPLETED, ROLLBACK_ACTION, rollbackInstant)), HoodieRollbackMetadata.class); metadata.getPartitionMetadata().forEach((key, value) -> Stream .concat(value.getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)), 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 4b322de5caf9c..94307bc972cea 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 @@ -77,7 +77,7 @@ public String writeAmplificationStats( TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); for (HoodieInstant instantTime : timeline.getInstants()) { String waf = "0"; - HoodieCommitMetadata commit = layout.getCommitMetadataSerDe().deserialize(instantTime, activeTimeline.getInstantDetails(instantTime).get(), + HoodieCommitMetadata commit = layout.getCommitMetadataSerDe().deserialize(instantTime, activeTimeline.getInstantContentStream(instantTime), HoodieCommitMetadata.class); if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format((float) commit.fetchTotalRecordsWritten() / commit.fetchTotalUpdateRecordsWritten()); 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 5244285c59315..88205c4bf7098 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 @@ -310,11 +310,11 @@ private String getInstantToRollback(HoodieTimeline timeline, HoodieInstant insta HoodieInstant instantToUse = HoodieCLI.getTableMetaClient().createNewInstant( HoodieInstant.State.REQUESTED, instant.getAction(), instant.requestedTime()); HoodieRollbackPlan metadata = TimelineMetadataUtils - .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class); + .deserializeAvroMetadata(timeline.getInstantContentStream(instantToUse), HoodieRollbackPlan.class); return metadata.getInstantToRollback().getCommitTime(); } else { HoodieRollbackMetadata metadata = TimelineMetadataUtils - .deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + .deserializeAvroMetadata(timeline.getInstantContentStream(instant), HoodieRollbackMetadata.class); return String.join(",", metadata.getCommitsRollback()); } } catch (IOException e) { @@ -335,12 +335,12 @@ private Map> getRolledBackInstantInfo(HoodieTimeline timeli HoodieInstant instantToUse = HoodieCLI.getTableMetaClient().createNewInstant( HoodieInstant.State.REQUESTED, rollbackInstant.getAction(), rollbackInstant.requestedTime()); HoodieRollbackPlan metadata = TimelineMetadataUtils - .deserializeAvroMetadata(timeline.getInstantDetails(instantToUse).get(), HoodieRollbackPlan.class); + .deserializeAvroMetadata(timeline.getInstantContentStream(instantToUse), HoodieRollbackPlan.class); rollbackInfoMap.computeIfAbsent(metadata.getInstantToRollback().getCommitTime(), k -> new ArrayList<>()) .add(rollbackInstant.requestedTime()); } else { HoodieRollbackMetadata metadata = TimelineMetadataUtils - .deserializeAvroMetadata(timeline.getInstantDetails(rollbackInstant).get(), HoodieRollbackMetadata.class); + .deserializeAvroMetadata(timeline.getInstantContentStream(rollbackInstant), HoodieRollbackMetadata.class); metadata.getCommitsRollback().forEach(instant -> { rollbackInfoMap.computeIfAbsent(instant, k -> new ArrayList<>()) .add(rollbackInstant.requestedTime()); 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 cb1b4b42172a7..36d2f5d8b7741 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 @@ -41,7 +41,7 @@ public static long countNewRecords(HoodieTableMetaClient metaClient, List { try { HoodieRestoreMetadata metadata = TimelineMetadataUtils - .deserializeAvroMetadata(activeTimeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class); + .deserializeAvroMetadata(activeTimeline.getInstantContentStream(instant), HoodieRestoreMetadata.class); metadata.getInstantsToRollback().forEach(c -> { Comparable[] row = new Comparable[4]; row[0] = metadata.getStartRestoreTime(); @@ -182,7 +182,7 @@ public void testShowRestore() throws IOException { // get metadata of instant HoodieRestoreMetadata instantMetadata = TimelineMetadataUtils.deserializeAvroMetadata( - activeTimeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class); + activeTimeline.getInstantContentStream(instant), HoodieRestoreMetadata.class); // generate expected result TableHeader header = new TableHeader() 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 5641c2c4aa059..674321aeb5c57 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 @@ -139,7 +139,7 @@ public void testShowRollbacks() { try { // get pair of rollback time and instant time HoodieRollbackMetadata metadata = TimelineMetadataUtils - .deserializeAvroMetadata(activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + .deserializeAvroMetadata(activeTimeline.getInstantContentStream(instant), HoodieRollbackMetadata.class); metadata.getCommitsRollback().forEach(c -> { Comparable[] row = new Comparable[5]; row[0] = metadata.getStartRollbackTime(); @@ -183,7 +183,7 @@ public void testShowRollback() throws IOException { List rows = new ArrayList<>(); // get metadata of instant HoodieRollbackMetadata metadata = TimelineMetadataUtils.deserializeAvroMetadata( - activeTimeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + activeTimeline.getInstantContentStream(instant), HoodieRollbackMetadata.class); // generate expect result metadata.getPartitionMetadata().forEach((key, value) -> Stream .concat(value.getSuccessDeleteFiles().stream().map(f -> Pair.of(f, true)), 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 3e28767b3e6ce..f28339bbebfdc 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 @@ -1425,7 +1425,7 @@ protected void setWriteSchemaForDeletes(HoodieTableMetaClient metaClient) { .lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstant.get(), - activeTimeline.getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + activeTimeline.getInstantContentStream(lastInstant.get()), HoodieCommitMetadata.class); String extraSchema = commitMetadata.getExtraMetadata().get(SCHEMA_KEY); if (!StringUtils.isNullOrEmpty(extraSchema)) { config.setSchema(commitMetadata.getExtraMetadata().get(SCHEMA_KEY)); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieColumnStatsIndexUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieColumnStatsIndexUtils.java index 238e64bac1abf..0ddf778d96b7c 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieColumnStatsIndexUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieColumnStatsIndexUtils.java @@ -63,7 +63,7 @@ public static void updateColsToIndex(HoodieTable dataTable, HoodieWriteConfig co HoodieInstant latestInstant = mdtMetaClient.getActiveTimeline().getWriteTimeline().filterCompletedInstants().lastInstant().get(); final HoodieCommitMetadata mdtCommitMetadata = mdtMetaClient.getTimelineLayout().getCommitMetadataSerDe().deserialize( latestInstant, - mdtMetaClient.getActiveTimeline().getInstantDetails(latestInstant).get(), + mdtMetaClient.getActiveTimeline().getInstantContentStream(latestInstant), HoodieCommitMetadata.class); if (mdtCommitMetadata.getPartitionToWriteStats().containsKey(MetadataPartitionType.COLUMN_STATS.getPartitionPath())) { // update data table's table config for list of columns indexed. 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 b504ce3c35e75..c57fa321cd4b7 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 @@ -47,6 +47,7 @@ import javax.annotation.Nonnull; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.Serializable; import java.util.ArrayList; @@ -103,8 +104,8 @@ private Pair> readInstant(GenericRecord record) { // should be json bytes. try { HoodieInstant instant = metaClient.getInstantGenerator().createNewInstant(HoodieInstant.State.COMPLETED, action, instantTime, stateTransitionTime); - org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata = new CommitMetadataSerDeV1().deserialize(instant, getUTF8Bytes(actionData.toString()), - org.apache.hudi.common.model.HoodieCommitMetadata.class); + org.apache.hudi.common.model.HoodieCommitMetadata commitMetadata = new CommitMetadataSerDeV1().deserialize( + instant, new ByteArrayInputStream(getUTF8Bytes(actionData.toString())), org.apache.hudi.common.model.HoodieCommitMetadata.class); // convert to avro bytes. return metaClient.getCommitMetadataSerDe().serialize(commitMetadata).get(); } catch (IOException e) { 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 4d3622e231ac4..6f59157b03957 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 @@ -1212,7 +1212,7 @@ public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) { HoodieRestorePlan restorePlan = null; try { restorePlan = TimelineMetadataUtils.deserializeAvroMetadata( - dataMetaClient.getActiveTimeline().readRestoreInfoAsBytes(requested).get(), HoodieRestorePlan.class); + dataMetaClient.getActiveTimeline().getInstantContentStream(requested), HoodieRestorePlan.class); } catch (IOException e) { throw new HoodieIOException(String.format("Deserialization of restore plan failed whose restore instant time is %s in data table", instantTime), 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 074a87cc63518..92ac35f8373f8 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 @@ -74,7 +74,7 @@ private int getCommitsSinceLastCleaning() { if (lastCleanInstant.isPresent() && !table.getActiveTimeline().isEmpty(lastCleanInstant.get())) { try { HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils - .deserializeHoodieCleanMetadata(table.getActiveTimeline().getInstantDetails(lastCleanInstant.get()).get()); + .deserializeHoodieCleanMetadata(table.getActiveTimeline().getInstantContentStream(lastCleanInstant.get())); String lastCompletedCommitTimestamp = cleanMetadata.getLastCompletedCommitTimestamp(); numCommits = commitTimeline.findInstantsAfter(lastCompletedCommitTimestamp).countInstants(); } catch (IOException e) { 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 893dfe8548a4a..ffb650ace3168 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 @@ -34,6 +34,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; 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.clean.CleanPlanV1MigrationHandler; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; @@ -143,7 +144,7 @@ private HoodieSavepointMetadata getSavepointMetadata(String savepointTimestamp) HoodieInstant instant = hoodieTable.getMetaClient().createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.SAVEPOINT_ACTION, savepointTimestamp); try { return TimelineMetadataUtils.deserializeHoodieSavepointMetadata( - hoodieTable.getActiveTimeline().getInstantDetails(instant).get()); + hoodieTable.getActiveTimeline().getInstantContentStream(instant)); } catch (IOException e) { throw new HoodieSavepointException("Could not get savepointed data files for savepoint " + savepointTimestamp, e); } @@ -187,7 +188,7 @@ private List getPartitionPathsForCleanByCommits(Option in hoodieTable.getActiveTimeline().deleteEmptyInstantIfExists(lastClean.get()); } else { HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils - .deserializeHoodieCleanMetadata(hoodieTable.getActiveTimeline().getInstantDetails(lastClean.get()).get()); + .deserializeHoodieCleanMetadata(hoodieTable.getActiveTimeline().getInstantContentStream(lastClean.get())); if ((cleanMetadata.getEarliestCommitToRetain() != null) && !cleanMetadata.getEarliestCommitToRetain().trim().isEmpty() && !hoodieTable.getActiveTimeline().getCommitsTimeline().isBeforeTimelineStarts(cleanMetadata.getEarliestCommitToRetain())) { @@ -251,12 +252,13 @@ private boolean isAnySavepointDeleted(HoodieCleanMetadata cleanMetadata) { private Stream getPartitionsForInstants(HoodieInstant instant) { try { if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) { - HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata.fromBytes( - hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); + HoodieReplaceCommitMetadata replaceCommitMetadata = layout.getCommitMetadataSerDe().deserialize( + instant, hoodieTable.getActiveTimeline().getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); return Stream.concat(replaceCommitMetadata.getPartitionToReplaceFileIds().keySet().stream(), replaceCommitMetadata.getPartitionToWriteStats().keySet().stream()); } else { HoodieCommitMetadata commitMetadata = hoodieTable.getMetaClient() - .getCommitMetadataSerDe().deserialize(instant, hoodieTable.getActiveTimeline().getInstantDetails(instant).get(), + .getCommitMetadataSerDe().deserialize(instant, hoodieTable.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); return commitMetadata.getPartitionToWriteStats().keySet().stream(); } 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 bdab87e6bb3f6..5c1f646b41a80 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 @@ -122,12 +122,12 @@ public void run() { break; case RESTORE_ACTION: HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( - metaClient.getActiveTimeline().getInstantDetails(instant).get()); + metaClient.getActiveTimeline().getInstantContentStream(instant)); metadataWriter.update(restoreMetadata, instant.requestedTime()); break; case ROLLBACK_ACTION: HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( - metaClient.getActiveTimeline().getInstantDetails(instant).get()); + metaClient.getActiveTimeline().getInstantContentStream(instant)); metadataWriter.update(rollbackMetadata, instant.requestedTime()); break; default: 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 cd71e71d3a874..eb60752ddeb20 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 @@ -53,7 +53,7 @@ public RecordBasedIndexingCatchupTask(HoodieTableMetadataWriter metadataWriter, @Override public void updateIndexForWriteAction(HoodieInstant instant) throws IOException { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); metadataWriter.update(commitMetadata, instant.requestedTime()); } } 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 6d475b76b04fd..56265bae434f5 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 @@ -53,7 +53,7 @@ public WriteStatBasedIndexingCatchupTask(HoodieTableMetadataWriter metadataWrite @Override public void updateIndexForWriteAction(HoodieInstant instant) throws IOException { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); metadataWriter.update(commitMetadata, instant.requestedTime()); } } 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 de0bebe6c5a5c..91016b0dd88ff 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 @@ -43,7 +43,7 @@ public static HoodieRestorePlan getRestorePlan(HoodieTableMetaClient metaClient, throws IOException { final HoodieInstant requested = metaClient.getInstantGenerator().getRollbackRequestedInstant(restoreInstant); return TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getActiveTimeline().readRestoreInfoAsBytes(requested).get(), HoodieRestorePlan.class); + metaClient.getActiveTimeline().getInstantContentStream(requested), HoodieRestorePlan.class); } public static String getSavepointToRestoreTimestampV1Schema(HoodieTable table, HoodieRestorePlan plan) { 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 1d7d5334345b8..26e793e2a4bc8 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 @@ -61,7 +61,7 @@ public static HoodieRollbackPlan getRollbackPlan(HoodieTableMetaClient metaClien // TODO: add upgrade step if required. final HoodieInstant requested = metaClient.getInstantGenerator().getRollbackRequestedInstant(rollbackInstant); return TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getActiveTimeline().readRollbackInfoAsBytes(requested).get(), HoodieRollbackPlan.class); + metaClient.getActiveTimeline().getInstantContentStream(requested), HoodieRollbackPlan.class); } static Map generateHeader(String instantToRollback, String rollbackInstantTime) { 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 30ec3faf541e9..8683e64facce7 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 @@ -81,12 +81,12 @@ public HoodieSavepointMetadata execute() { try { if (instant.isCompleted()) { return deserializeHoodieCleanMetadata( - table.getActiveTimeline().getInstantDetails(instant).get()) + table.getActiveTimeline().getInstantContentStream(instant)) .getEarliestCommitToRetain(); } else { // clean is pending or inflight return deserializeCleanerPlan( - table.getActiveTimeline().getInstantDetails(instantGenerator.createNewInstant(REQUESTED, instant.getAction(), instant.requestedTime())).get()) + table.getActiveTimeline().getInstantContentStream(instantGenerator.createNewInstant(REQUESTED, instant.getAction(), instant.requestedTime()))) .getEarliestInstantToRetain().getTimestamp(); } } catch (IOException e) { 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 b0016de2adfb2..3cd8ac5eecadc 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 @@ -90,20 +90,23 @@ public static Option> getBaseAndLogFilePathsFromTimeline( switch (instant.getAction()) { case COMMIT_ACTION: - case DELTA_COMMIT_ACTION: + case DELTA_COMMIT_ACTION: { TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); final HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize(instant, - timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + timeline.getInstantContentStream(instant), HoodieCommitMetadata.class); return Option.of(commitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream) .map(HoodieWriteStat::getPath).collect(Collectors.toSet())); + } case REPLACE_COMMIT_ACTION: - case CLUSTERING_ACTION: + case CLUSTERING_ACTION: { + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); final HoodieReplaceCommitMetadata replaceCommitMetadata = - HoodieReplaceCommitMetadata.fromBytes( - timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + layout.getCommitMetadataSerDe().deserialize(instant, + timeline.getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); return Option.of(replaceCommitMetadata.getPartitionToWriteStats().values().stream().flatMap(List::stream) .map(HoodieWriteStat::getPath).collect(Collectors.toSet())); + } default: return Option.empty(); } 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 52bbc830238a4..3682a5f1e1769 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 @@ -305,9 +305,10 @@ static boolean rewriteTimelineV2InstantFileToV1Format(HoodieInstant instant, Hoo || ((instant.getAction().equals(REPLACE_COMMIT_ACTION) || instant.getAction().equals(CLUSTERING_ACTION)) && instant.isCompleted())) { Option data; if (instant.getAction().equals(REPLACE_COMMIT_ACTION) || instant.getAction().equals(CLUSTERING_ACTION)) { - data = commitMetadataSerDeV1.serialize(HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class)); + data = commitMetadataSerDeV1.serialize(commitMetadataSerDeV1.deserialize( + instant, metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieReplaceCommitMetadata.class)); } else { - data = commitMetadataSerDeV1.serialize(commitMetadataSerDeV2.deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class)); + data = commitMetadataSerDeV1.serialize(commitMetadataSerDeV2.deserialize(instant, metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class)); } String toPathStr = toPath.toUri().toString(); activeTimelineV1.createFileInMetaPath(toPathStr, data, true); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java index 3affa7b64127e..1801b6755f6df 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/table/upgrade/SevenToEightUpgradeHandler.java @@ -289,7 +289,7 @@ static boolean rewriteTimelineV1InstantFileToV2Format(HoodieInstant instant, Hoo boolean success = true; if (instant.getAction().equals(COMMIT_ACTION) || instant.getAction().equals(DELTA_COMMIT_ACTION) || (instant.getAction().equals(REPLACE_COMMIT_ACTION) && instant.isCompleted())) { Class clazz = instant.getAction().equals(REPLACE_COMMIT_ACTION) ? HoodieReplaceCommitMetadata.class : HoodieCommitMetadata.class; - HoodieCommitMetadata commitMetadata = commitMetadataSerDeV1.deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), clazz); + HoodieCommitMetadata commitMetadata = commitMetadataSerDeV1.deserialize(instant, metaClient.getActiveTimeline().getInstantContentStream(instant), clazz); Option data = commitMetadataSerDeV2.serialize(commitMetadata); String toPathStr = toPath.toUri().toString(); activeTimelineV2.createFileInMetaPath(toPathStr, data, true); 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 0dc9dd398949f..af0b5e7bae2d5 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 @@ -431,7 +431,7 @@ protected Pair getPartitionAndBaseFilePathsFromLatestCommitMetad String extension = metaClient.getTableConfig().getBaseFileFormat().getFileExtension(); HoodieInstant instant = metaClient.getCommitsTimeline().filterCompletedInstants().lastInstant().get(); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); String filePath = commitMetadata.getPartitionToWriteStats().values().stream() .flatMap(Collection::stream).filter(s -> s.getPath().endsWith(extension)).findAny() .map(HoodieWriteStat::getPath).orElse(null); @@ -554,9 +554,10 @@ protected void verifyClusteredFilesWithReplaceCommitMetadata(String partitionPat metaClient = HoodieTableMetaClient.reload(createMetaClient()); HoodieInstant replaceCommitInstant = metaClient.getActiveTimeline().getCompletedReplaceTimeline().firstInstant().get(); - HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(replaceCommitInstant).get(), - HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata replaceCommitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + replaceCommitInstant, + metaClient.getActiveTimeline().getInstantContentStream(replaceCommitInstant), + HoodieReplaceCommitMetadata.class); List filesFromReplaceCommit = new ArrayList<>(); replaceCommitMetadata.getPartitionToWriteStats() @@ -881,14 +882,14 @@ protected void testCommitWritesRelativePaths(Function transformInputFn) throws E HoodieInstant commitInstant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, actionType, instantTime); HoodieTimeline commitTimeline = metaClient.getCommitTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(commitInstant, commitTimeline.getInstantDetails(commitInstant).get(), HoodieCommitMetadata.class); + .deserialize(commitInstant, commitTimeline.getInstantContentStream(commitInstant), HoodieCommitMetadata.class); StoragePath basePath = metaClient.getBasePath(); Collection commitPathNames = commitMetadata.getFileIdAndFullPaths(basePath).values(); // Read from commit file HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime); HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + metaClient.reloadActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); HashMap paths = metadata.getFileIdAndFullPaths(basePath); // Compare values in both to make sure they are equal. @@ -913,7 +914,7 @@ protected void testMetadataStatsOnCommit(boolean populateMetaFields, Function tr // Read from commit file HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime0); HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - createMetaClient().reloadActiveTimeline().getInstantDetails(instant).get(), + createMetaClient().reloadActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -934,7 +935,7 @@ protected void testMetadataStatsOnCommit(boolean populateMetaFields, Function tr instant = INSTANT_GENERATOR.createNewInstant(COMPLETED, COMMIT_ACTION, instantTime1); // Read from commit file metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + metaClient.reloadActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); inserts = 0; int upserts = 0; 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 7b27a1ea0168c..ead0ebddae5ad 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 @@ -56,6 +56,7 @@ import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -195,7 +196,7 @@ public void testArchivedClean() throws Exception { assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.CLEAN_ACTION, archived.getAction()); assertDoesNotThrow(() -> CleanerUtils.getCleanerMetadata(metaClient, archived.getMetadata().array())); - assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeCleanerPlan(archived.getPlan().array())); + assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeCleanerPlan(new ByteArrayInputStream(archived.getPlan().array()))); } @Test @@ -206,7 +207,6 @@ public void testArchivedReplace() throws Exception { HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived.getAction()); - assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieReplaceCommitMetadata.class)); assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeRequestedReplaceMetadata(archived.getPlan().array())); } @@ -218,10 +218,9 @@ public void testArchivedInsertOverwriteWithoutClustering() throws Exception { HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived.getAction()); - assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieReplaceCommitMetadata.class)); 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"); + new ByteArrayInputStream(archived.getPlan().array()), HoodieCommitMetadata.class), "Insert overwrite without clustering should have a plan"); String newCommitTime2 = HoodieTestTable.makeNewCommitTime(); createReplace(newCommitTime2, WriteOperationType.INSERT_OVERWRITE_TABLE, false); @@ -229,10 +228,9 @@ public void testArchivedInsertOverwriteWithoutClustering() throws Exception { HoodieLSMTimelineInstant archived2 = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime2), metaClient); assertEquals(newCommitTime2, archived2.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived2.getAction()); - assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived2.getMetadata().array(), HoodieReplaceCommitMetadata.class)); assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime2), - archived2.getPlan().array(), HoodieCommitMetadata.class), + new ByteArrayInputStream(archived2.getPlan().array()), HoodieCommitMetadata.class), "Insert overwrite table without clustering should have a plan"); } @@ -244,7 +242,6 @@ public void testArchivedInsertOverwriteWithClustering() throws Exception { HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); assertEquals(newCommitTime, archived.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived.getAction()); - assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieReplaceCommitMetadata.class)); assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeRequestedReplaceMetadata(archived.getPlan().array())); String newCommitTime2 = HoodieTestTable.makeNewCommitTime(); @@ -253,7 +250,6 @@ public void testArchivedInsertOverwriteWithClustering() throws Exception { HoodieLSMTimelineInstant archived2 = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime2), metaClient); assertEquals(newCommitTime2, archived2.getInstantTime()); assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, archived2.getAction()); - assertDoesNotThrow(() -> HoodieReplaceCommitMetadata.fromBytes(archived2.getMetadata().array(), HoodieReplaceCommitMetadata.class)); assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeRequestedReplaceMetadata(archived2.getPlan().array())); } @@ -266,7 +262,7 @@ public void testArchivedCommit() throws Exception { assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), - archived.getMetadata().array(), HoodieCommitMetadata.class)); + new ByteArrayInputStream(archived.getMetadata().array()), HoodieCommitMetadata.class)); } @Test @@ -299,7 +295,7 @@ public void testArchivedCompaction() throws Exception { assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); assertDoesNotThrow(() -> metaClient.getCommitMetadataSerDe().deserialize( INSTANT_GENERATOR.createNewInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, newCommitTime), - archived.getMetadata().array(), HoodieCommitMetadata.class)); + new ByteArrayInputStream(archived.getMetadata().array()), HoodieCommitMetadata.class)); assertDoesNotThrow(() -> CompactionUtils.getCompactionPlan(metaClient, Option.of(archived.getPlan().array()))); } 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 4edf816dd472d..5165af7ed797d 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 @@ -121,7 +121,7 @@ private List getPartitionPathInActiveTimeline(HoodieTable { try { HoodieCommitMetadata metadata = hoodieTable.getMetaClient().getCommitMetadataSerDe().deserialize(instant, - cowCommitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + cowCommitTimeline.getInstantContentStream(instant), HoodieCommitMetadata.class); partitions.addAll(metadata.getWritePartitionPaths()); } catch (IOException e) { // ignore Exception here diff --git a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java index 04b5527474482..b94d05a6b9fe0 100644 --- a/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java +++ b/hudi-client/hudi-java-client/src/main/java/org/apache/hudi/table/action/commit/JavaUpsertPartitioner.java @@ -323,7 +323,7 @@ protected static long averageBytesPerRecord(HoodieTimeline commitTimeline, Hoodi HoodieInstant instant = instants.next(); TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() - .deserialize(instant, commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, commitTimeline.getInstantContentStream(instant), HoodieCommitMetadata.class); long totalBytesWritten = commitMetadata.fetchTotalBytesWritten(); long totalRecordsWritten = commitMetadata.fetchTotalRecordsWritten(); if (totalBytesWritten > fileSizeThreshold && totalRecordsWritten > 0) { 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 9992441649ee2..17fb3e658761b 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 @@ -1441,7 +1441,7 @@ public void testColStatsPrefixLookup() throws IOException { metaClient.getActiveTimeline().getInstants().forEach(entry -> { try { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(entry, metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); + .deserialize(entry, metaClient.getActiveTimeline().getInstantContentStream(entry), HoodieCommitMetadata.class); String commitTime = entry.requestedTime(); if (!commitToPartitionsToFiles.containsKey(commitTime)) { commitToPartitionsToFiles.put(commitTime, new HashMap<>()); 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 650eb12326df7..7e51575ebf93f 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 @@ -949,7 +949,7 @@ private static HashMap getLatestFileIDsToFullPath(String basePat TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); for (HoodieInstant commit : commitsToReturn) { HoodieCommitMetadata metadata = - layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantContentStream(commit), HoodieCommitMetadata.class); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(new StoragePath(basePath))); } return fileIdToFullPath; 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 8873fb01c97b1..576346c63b9c9 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 @@ -57,7 +57,7 @@ static long averageBytesPerRecord(HoodieTimeline commitTimeline, HoodieWriteConf HoodieInstant instant = instants.next(); try { HoodieCommitMetadata commitMetadata = commitMetadataSerDe - .deserialize(instant, commitTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, commitTimeline.getInstantContentStream(instant), 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/test/java/org/apache/hudi/client/TestSavepoint.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/TestSavepoint.java index 7163df31ad690..fecaa44a93ef5 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 @@ -107,13 +107,13 @@ public void testSavepoint(boolean enableMetadataTable, Map savepointPartitionMetadataMap = TimelineMetadataUtils.deserializeHoodieSavepointMetadata( - savepointTimeline.getInstantDetails(savepointTimeline.firstInstant().get()).get()) + savepointTimeline.getInstantContentStream(savepointTimeline.firstInstant().get())) .getPartitionMetadata(); HoodieTimeline commitsTimeline = table.getActiveTimeline().getCommitsTimeline(); Map> partitionToWriteStats = metaClient.getCommitMetadataSerDe().deserialize( commitsTimeline.lastInstant().get(), - commitsTimeline.getInstantDetails(commitsTimeline.lastInstant().get()).get(), + commitsTimeline.getInstantContentStream(commitsTimeline.lastInstant().get()), HoodieCommitMetadata.class) .getPartitionToWriteStats(); 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 798dc0bdc8c3e..4b7e3cec7a76b 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 @@ -371,7 +371,7 @@ private int getNumCompactions(HoodieTableMetaClient metaClient) { try { return s.getAction().equals(HoodieTimeline.COMMIT_ACTION) && metaClient.getCommitMetadataSerDe().deserialize(s, - timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class) + timeline.getInstantContentStream(s), HoodieCommitMetadata.class) .getOperationType().equals(COMPACT); } catch (IOException e) { throw new RuntimeException(e); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java index fa8231bbce3e9..fef9c38f247bc 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/TestCleaner.java @@ -269,7 +269,7 @@ private void testInsertAndCleanFailedWritesByVersions( Option rollBackInstantForFailedCommit = timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.getInstantDetails(rollBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + timeline.getInstantContentStream(rollBackInstantForFailedCommit.get()), HoodieRollbackMetadata.class); // Rollback of one of the failed writes should have deleted 3 files assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); } @@ -546,7 +546,7 @@ private void testFailedInsertAndCleanByCommits( Option rollBackInstantForFailedCommit = timeline.getTimelineOfActions( CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants().lastInstant(); HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.getInstantDetails(rollBackInstantForFailedCommit.get()).get(), HoodieRollbackMetadata.class); + timeline.getInstantContentStream(rollBackInstantForFailedCommit.get()), HoodieRollbackMetadata.class); // Rollback of one of the failed writes should have deleted 3 files assertEquals(3, rollbackMetadata.getTotalFilesDeleted()); } 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 2515d46106852..6de95f5cf5242 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 @@ -216,7 +216,7 @@ private void validateFilesAfterCleaning( try { HoodieInstant instant1 = timeline.filter(inst -> inst.requestedTime().equals(newInstant)) .firstInstant().get(); - return layout.getCommitMetadataSerDe().deserialize(instant1, timeline.getInstantDetails(instant1).get(), HoodieCommitMetadata.class) + return layout.getCommitMetadataSerDe().deserialize(instant1, timeline.getInstantContentStream(instant1), HoodieCommitMetadata.class) .getWriteStats(); } catch (IOException e) { return Collections.EMPTY_LIST; 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 03291e82f536f..3e7cb947d9ed8 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 @@ -193,7 +193,7 @@ private void testInsertAndCleanByVersions( HashMap> fileIdToVersions = new HashMap<>(); for (HoodieInstant entry : timeline.getInstants()) { HoodieCommitMetadata commitMetadata = - metaClient.getCommitMetadataSerDe().deserialize(entry, timeline.getInstantDetails(entry).get(), HoodieCommitMetadata.class); + metaClient.getCommitMetadataSerDe().deserialize(entry, timeline.getInstantContentStream(entry), 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 971518a685fad..a9a4a77669cfd 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 @@ -36,6 +36,7 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -82,7 +83,7 @@ public void testAverageRecordSize(List>> in }); instants.add(hoodieInstant); try { - when(mockTimeline.getInstantDetails(hoodieInstant)).thenReturn(TimelineMetadataUtils.serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata)); + when(mockTimeline.getInstantContentStream(hoodieInstant)).thenReturn(new ByteArrayInputStream(TimelineMetadataUtils.serializeCommitMetadata(COMMIT_METADATA_SER_DE, commitMetadata).get())); } catch (IOException e) { throw new RuntimeException("Should not have failed", e); } 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 8995917621e80..a258536df2a5e 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 @@ -174,7 +174,7 @@ public void testAverageBytesPerRecordForNonEmptyCommitTimeLine() throws Exceptio when(commitTimeLine.empty()).thenReturn(false); when(commitTimeLine.getReverseOrderedInstants()).thenReturn(setupHoodieInstants().stream()); LinkedList> commits = generateCommitMetadataList(); - when(commitTimeLine.getInstantDetails(any(HoodieInstant.class))).thenAnswer(invocationOnMock -> commits.pop()); + when(commitTimeLine.getInstantContentStream(any(HoodieInstant.class))).thenAnswer(invocationOnMock -> commits.pop()); long expectAvgSize = (long) Math.ceil((1.0 * 7500) / 1500); 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/functional/TestCleanActionExecutor.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/table/functional/TestCleanActionExecutor.java index 463c453b6f531..75e7c16cbfe10 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 @@ -50,6 +50,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.EnumSource; +import java.io.ByteArrayInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.util.Collections; @@ -137,7 +138,7 @@ void testPartialCleanFailure(CleanFailureType failureType) throws IOException { HoodieActiveTimeline cleanTimeline = mock(HoodieActiveTimeline.class); when(activeTimeline.getCleanerTimeline()).thenReturn(cleanTimeline); when(cleanTimeline.getInstants()).thenReturn(Collections.singletonList(cleanInstant)); - when(activeTimeline.getInstantDetails(cleanInstant)).thenReturn(TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); + when(activeTimeline.getInstantContentStream(cleanInstant)).thenReturn(new ByteArrayInputStream(TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan).get())); when(activeTimeline.readCleanerInfoAsBytes(cleanInstant)).thenReturn(TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); when(mockHoodieTable.getCleanTimeline()).thenReturn(cleanTimeline); 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 c305ccd42f093..037362b9c4af9 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 @@ -142,7 +142,7 @@ private static HashMap getLatestFileIDsToFullPath(String basePat TimelineLayout layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion()); for (HoodieInstant commit : commitsToReturn) { HoodieCommitMetadata metadata = - layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + layout.getCommitMetadataSerDe().deserialize(commit, commitTimeline.getInstantContentStream(commit), HoodieCommitMetadata.class); fileIdToFullPath.putAll(metadata.getFileIdAndFullPaths(new StoragePath(basePath))); } return fileIdToFullPath; @@ -328,8 +328,7 @@ public static HoodieTableMetaClient createMetaClient(SparkSession spark, String public static Option getCommitMetadataForInstant(HoodieTableMetaClient metaClient, HoodieInstant instant) { try { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - byte[] data = timeline.getInstantDetails(instant).get(); - return Option.of(metaClient.getCommitMetadataSerDe().deserialize(instant, data, HoodieCommitMetadata.class)); + return Option.of(metaClient.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantContentStream(instant), HoodieCommitMetadata.class)); } catch (Exception e) { throw new HoodieException("Failed to read schema from commit metadata", e); } 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 2f42472c2b14f..a8201fbe61e3d 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 @@ -32,6 +32,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.InputStream; import java.io.Serializable; import java.util.ArrayList; import java.util.Collection; @@ -43,6 +44,7 @@ import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeCommitMetadata; + /** * All the metadata that gets stored along with a commit. * ******** IMPORTANT ******** @@ -250,9 +252,9 @@ public static T fromJsonString(String jsonStr, Class clazz) throws Except * parse the bytes of deltacommit, and get the base file and the log files belonging to this * provided file group. */ - public static Option>> getFileSliceForFileGroupFromDeltaCommit(byte[] bytes, HoodieFileGroupId fileGroupId) { + public static Option>> getFileSliceForFileGroupFromDeltaCommit(InputStream inputStream, HoodieFileGroupId fileGroupId) { try { - org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadata = deserializeCommitMetadata(bytes); + org.apache.hudi.avro.model.HoodieCommitMetadata commitMetadata = deserializeCommitMetadata(inputStream); Map> partitionToWriteStatsMap = commitMetadata.getPartitionToWriteStats(); for (Map.Entry> partitionToWriteStat: partitionToWriteStatsMap.entrySet()) { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java index 401d958799e59..0e13f21d3484a 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java @@ -18,29 +18,23 @@ package org.apache.hudi.common.model; -import org.apache.hudi.common.util.JsonUtils; - import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.apache.hudi.common.util.JsonUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.nio.charset.StandardCharsets; import java.util.ArrayList; import java.util.HashMap; import java.util.List; import java.util.Map; -import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertCommitMetadataToJsonBytes; -import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeReplaceCommitMetadata; -import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes; - /** * All the metadata that gets stored along with a commit. * ******** IMPORTANT ******** * For any newly added/removed data fields, make sure we have the same definition in * src/main/avro/HoodieReplaceCommitMetadata.avsc file!!!!! - * + *

* For any newly added subclass, make sure we add corresponding handler in * org.apache.hudi.common.table.timeline.versioning.v2.CommitMetadataSerDeV2#deserialize method. * *************************** @@ -119,23 +113,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(deserializeReplaceCommitMetadata(bytes), org.apache.hudi.avro.model.HoodieReplaceCommitMetadata.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 "HoodieReplaceMetadata{" + "partitionToWriteStats=" + partitionToWriteStats 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 315e8ff67b902..636e8101900ea 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 @@ -267,7 +267,7 @@ private Option getTableSchemaFromLatestCommitMetadataForClustering(boole try { return !StringUtils.isNullOrEmpty(metaClient.getCommitMetadataSerDe().deserialize( s, - reversedTimeline.getInstantDetails(s).get(), + reversedTimeline.getInstantContentStream(s), HoodieCommitMetadata.class) .getMetadata(HoodieCommitMetadata.SCHEMA_KEY)); } catch (IOException e) { @@ -366,7 +366,7 @@ public Schema readSchemaFromLastCompaction(Option lastCompactionC // Read from the compacted file wrote HoodieCommitMetadata compactionMetadata = metaClient.getCommitMetadataSerDe().deserialize( - lastCompactionCommit, activeTimeline.getInstantDetails(lastCompactionCommit).get(),HoodieCommitMetadata.class); + lastCompactionCommit, activeTimeline.getInstantContentStream(lastCompactionCommit), 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())); @@ -411,7 +411,7 @@ public Option getTableInternalSchemaFromCommitMetadata() { .filter(instant -> { // consider only instants that can update/change schema. try { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( - instant, completedInstants.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + instant, completedInstants.getInstantContentStream(instant), HoodieCommitMetadata.class); return WriteOperationType.canUpdateSchema(commitMetadata.getOperationType()); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); @@ -522,9 +522,9 @@ private HoodieCommitMetadata getCachedCommitMetadata(HoodieInstant instant) { return commitMetadataCache.get() .computeIfAbsent(instant, (missingInstant) -> { HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); - byte[] data = timeline.getInstantDetails(missingInstant).get(); try { - return metaClient.getCommitMetadataSerDe().deserialize(missingInstant, data, HoodieCommitMetadata.class); + return metaClient.getCommitMetadataSerDe().deserialize( + missingInstant, timeline.getInstantContentStream(missingInstant), 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 979af5dee9597..4e60027a05381 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 @@ -330,7 +330,7 @@ private Option getDependentFileSliceForLogFile( String currentLogFileName = new StoragePath(currentLogFile).getName(); Option>> fileSliceOpt = HoodieCommitMetadata.getFileSliceForFileGroupFromDeltaCommit( - metaClient.getActiveTimeline().getInstantDetails(instant).get(), fgId); + metaClient.getActiveTimeline().getInstantContentStream(instant), fgId); if (fileSliceOpt.isPresent()) { Pair> fileSlice = fileSliceOpt.get(); try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java index b1f5ea6f1e189..a0b24125e5266 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/BaseHoodieTimeline.java @@ -20,22 +20,14 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.JsonUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; -import com.fasterxml.jackson.databind.exc.MismatchedInputException; -import org.apache.avro.file.DataFileStream; -import org.apache.avro.io.DatumReader; -import org.apache.avro.specific.SpecificDatumReader; -import org.apache.avro.specific.SpecificRecord; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.IOException; import java.io.InputStream; import java.io.Serializable; import java.security.MessageDigest; @@ -567,27 +559,6 @@ public InputStream getInstantContentStream(HoodieInstant instant) { return getInstantReader().getContentStream(instant); } - @Override - public T deserializeInstantContent(HoodieInstant instant, Class clazz) throws IOException { - if (SpecificRecord.class.isAssignableFrom(clazz)) { - try (InputStream inputStream = getInstantContentStream(instant)) { - DatumReader reader = new SpecificDatumReader<>(clazz); - DataFileStream fileReader = new DataFileStream<>(inputStream, reader); - ValidationUtils.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz); - return fileReader.next(); - } - } else { - try (InputStream inputStream = getInstantContentStream(instant)) { - return JsonUtils.getObjectMapper().readValue(inputStream, clazz); - } catch (MismatchedInputException ex) { - if (ex.getMessage().startsWith("No content to map")) { - return ReflectionUtils.loadClass(clazz.getName()); - } - throw ex; - } - } - } - @Override public boolean isEmpty(HoodieInstant instant) { return getInstantDetails(instant).get().length == 0; 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 index ba2cb57db72a3..0188995059d80 100644 --- 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 @@ -22,6 +22,7 @@ import org.apache.hudi.common.util.Option; import java.io.IOException; +import java.io.InputStream; import java.io.Serializable; /** @@ -29,7 +30,7 @@ */ public interface CommitMetadataSerDe extends Serializable { - T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException; + T deserialize(HoodieInstant instant, InputStream instantStream, 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/HoodieInstantReader.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantReader.java index 0df02fec1ae68..02931d7fe5f2d 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstantReader.java @@ -39,6 +39,7 @@ default InputStream getContentStream(HoodieInstant instant) { } /** + * dd * Reads the provided instant's content into a byte array for parsing. * @param instant the instant to read * @return an InputStream with the details 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 06c0f9883f1ee..3abfce51b7a67 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 @@ -23,7 +23,6 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.StringUtils; -import java.io.IOException; import java.io.InputStream; import java.io.Serializable; import java.util.List; @@ -457,8 +456,6 @@ public interface HoodieTimeline extends HoodieInstantReader, Serializable { */ InputStream getInstantContentStream(HoodieInstant instant); - T deserializeInstantContent(HoodieInstant instant, Class clazz) throws IOException; - boolean isEmpty(HoodieInstant instant); /** 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 23c3d276101c4..89441e7c1a016 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 @@ -48,8 +48,10 @@ import org.apache.avro.specific.SpecificRecord; import org.apache.avro.specific.SpecificRecordBase; +import java.io.BufferedInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.nio.ByteBuffer; /** @@ -63,24 +65,25 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst archivedMetaWrapper.setCommitTime(hoodieInstant.requestedTime()); archivedMetaWrapper.setActionState(hoodieInstant.getState().name()); archivedMetaWrapper.setStateTransitionTime(hoodieInstant.getCompletionTime()); + CommitMetadataSerDe serDe = metaClient.getCommitMetadataSerDe(); switch (hoodieInstant.getAction()) { case HoodieTimeline.CLEAN_ACTION: { if (hoodieInstant.isCompleted()) { - archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get())); + archivedMetaWrapper.setHoodieCleanMetadata(CleanerUtils.getCleanerMetadata(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant))); } else { - archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get())); + archivedMetaWrapper.setHoodieCleanerPlan(CleanerUtils.getCleanerPlan(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant))); } archivedMetaWrapper.setActionType(ActionType.clean.name()); break; } case HoodieTimeline.COMMIT_ACTION: { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class) .ifPresent(commitMetadata -> archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata))); archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } case HoodieTimeline.DELTA_COMMIT_ACTION: { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class) .ifPresent(deltaCommitMetadata -> archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(deltaCommitMetadata))); archivedMetaWrapper.setActionType(ActionType.deltacommit.name()); break; @@ -88,20 +91,20 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst case HoodieTimeline.REPLACE_COMMIT_ACTION: case HoodieTimeline.CLUSTERING_ACTION: { if (hoodieInstant.isCompleted()) { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieReplaceCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieReplaceCommitMetadata.class) .ifPresent(replaceCommitMetadata -> archivedMetaWrapper.setHoodieReplaceCommitMetadata(convertReplaceCommitMetadata(replaceCommitMetadata))); } else if (hoodieInstant.isInflight()) { // 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 = getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class); + Option inflightCommitMetadata = getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class); if (inflightCommitMetadata.isPresent()) { archivedMetaWrapper.setHoodieInflightReplaceMetadata(convertCommitMetadata(inflightCommitMetadata.get())); } } else { // we may have cases with empty HoodieRequestedReplaceMetadata e.g. insert_overwrite_table or insert_overwrite // without clustering. However, we should revisit the requested commit file standardization - Option requestedReplaceMetadata = getRequestedReplaceMetadata(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant)); + Option requestedReplaceMetadata = getRequestedReplaceMetadata(metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant)); if (requestedReplaceMetadata.isPresent()) { archivedMetaWrapper.setHoodieRequestedReplaceMetadata(requestedReplaceMetadata.get()); } @@ -113,20 +116,20 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst case HoodieTimeline.ROLLBACK_ACTION: { if (hoodieInstant.isCompleted()) { archivedMetaWrapper.setHoodieRollbackMetadata( - TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); + TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieRollbackMetadata.class)); } archivedMetaWrapper.setActionType(ActionType.rollback.name()); break; } case HoodieTimeline.SAVEPOINT_ACTION: { archivedMetaWrapper.setHoodieSavePointMetadata( - TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); + TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieSavepointMetadata.class)); archivedMetaWrapper.setActionType(ActionType.savepoint.name()); break; } case HoodieTimeline.COMPACTION_ACTION: { if (hoodieInstant.isRequested()) { - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant)); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant)); archivedMetaWrapper.setHoodieCompactionPlan(plan); } archivedMetaWrapper.setActionType(ActionType.compaction.name()); @@ -134,7 +137,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst } case HoodieTimeline.LOG_COMPACTION_ACTION: { if (hoodieInstant.isRequested()) { - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant)); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant)); archivedMetaWrapper.setHoodieCompactionPlan(plan); } archivedMetaWrapper.setActionType(ActionType.logcompaction.name()); @@ -186,7 +189,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper( break; } case HoodieTimeline.COMMIT_ACTION: { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class) .ifPresent(commitMetadata -> archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata))); archivedMetaWrapper.setActionType(ActionType.commit.name()); @@ -198,7 +201,7 @@ public static HoodieArchivedMetaEntry createMetaWrapper( break; } case HoodieTimeline.DELTA_COMMIT_ACTION: { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class) .ifPresent(commitMetadata -> archivedMetaWrapper.setHoodieCommitMetadata(convertCommitMetadata(commitMetadata))); archivedMetaWrapper.setActionType(ActionType.deltacommit.name()); @@ -211,13 +214,13 @@ public static HoodieArchivedMetaEntry createMetaWrapper( } case HoodieTimeline.REPLACE_COMMIT_ACTION: case HoodieTimeline.CLUSTERING_ACTION: { - getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieReplaceCommitMetadata.class) + getCommitMetadata(metaClient, hoodieInstant, HoodieReplaceCommitMetadata.class) .ifPresent(replaceCommitMetadata -> archivedMetaWrapper.setHoodieReplaceCommitMetadata(convertReplaceCommitMetadata(replaceCommitMetadata))); // 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 = getCommitMetadata(metaClient.getActiveTimeline(), hoodieInstant, HoodieCommitMetadata.class); + Option inflightCommitMetadata = getCommitMetadata(metaClient, hoodieInstant, HoodieCommitMetadata.class); if (inflightCommitMetadata.isPresent()) { archivedMetaWrapper.setHoodieInflightReplaceMetadata(convertCommitMetadata(inflightCommitMetadata.get())); } @@ -227,25 +230,25 @@ public static HoodieArchivedMetaEntry createMetaWrapper( } case HoodieTimeline.ROLLBACK_ACTION: { archivedMetaWrapper.setHoodieRollbackMetadata( - TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); + TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieRollbackMetadata.class)); archivedMetaWrapper.setActionType(ActionType.rollback.name()); break; } case HoodieTimeline.SAVEPOINT_ACTION: { archivedMetaWrapper.setHoodieSavePointMetadata( - TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieSavepointMetadata.class)); + TimelineMetadataUtils.deserializeAvroMetadata(metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieSavepointMetadata.class)); archivedMetaWrapper.setActionType(ActionType.savepoint.name()); break; } case HoodieTimeline.COMPACTION_ACTION: { // should be handled by commit_action branch though, this logic is redundant. - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant)); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant)); archivedMetaWrapper.setHoodieCompactionPlan(plan); archivedMetaWrapper.setActionType(ActionType.compaction.name()); break; } case HoodieTimeline.LOG_COMPACTION_ACTION: { - HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantDetails(hoodieInstant)); + HoodieCompactionPlan plan = CompactionUtils.getCompactionPlan(metaClient, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant)); archivedMetaWrapper.setHoodieCompactionPlan(plan); archivedMetaWrapper.setActionType(ActionType.logcompaction.name()); break; @@ -339,8 +342,8 @@ public static HoodieArchivedMetaEntry createMetaWrapperForEmptyInstant(HoodieIns return archivedMetaWrapper; } - private static Option getCommitMetadata(HoodieActiveTimeline timeline, HoodieInstant instant, Class clazz) throws IOException { - T commitMetadata = timeline.deserializeInstantContent(instant, clazz); + private static Option getCommitMetadata(HoodieTableMetaClient metaClient, HoodieInstant instant, Class clazz) throws IOException { + T commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, metaClient.getActiveTimeline().getInstantReader().getContentStream(instant), clazz); // an empty file will return the default instance with an UNKNOWN operation type and in that case we return an empty option if (commitMetadata.getOperationType() == WriteOperationType.UNKNOWN) { return Option.empty(); @@ -348,15 +351,9 @@ private static Option getCommitMetadata(Hood return Option.of(commitMetadata); } - private static Option getRequestedReplaceMetadata(Option requestedContent) throws IOException { - if (!requestedContent.isPresent() || requestedContent.get().length == 0) { - // requested commit files can be empty in some certain cases, e.g. insert_overwrite or insert_overwrite_table. - // However, it appears requested files are supposed to contain meta data and we should revisit the standardization - // of requested commit files - // TODO revisit requested commit file standardization https://issues.apache.org/jira/browse/HUDI-1739 - return Option.empty(); - } - return Option.of(TimelineMetadataUtils.deserializeRequestedReplaceMetadata(requestedContent.get())); + private static Option getRequestedReplaceMetadata(InputStream inputStream) throws IOException { + // TODO revisit requested commit file standardization https://issues.apache.org/jira/browse/HUDI-1739 + return Option.of(TimelineMetadataUtils.deserializeRequestedReplaceMetadata(inputStream)); } public static Option getHoodieCommitMetadata(HoodieTableMetaClient metaClient, HoodieInstant hoodieInstant) throws IOException { @@ -418,4 +415,12 @@ public static byte[] convertCommitMetadataToJsonB throw new HoodieIOException("Failed to convert to JSON.", e); } } + + public static boolean isEmptyStream(InputStream inputStream) throws IOException { + BufferedInputStream bis = new BufferedInputStream(inputStream); + bis.mark(1); + boolean isEmpty = (bis.read() == -1); + bis.reset(); + return isEmpty; + } } 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 4e199ecb7d1ee..acba69c04382e 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 @@ -40,6 +40,7 @@ import org.apache.hudi.storage.StoragePathInfo; import org.apache.avro.file.DataFileReader; +import org.apache.avro.file.DataFileStream; import org.apache.avro.file.DataFileWriter; import org.apache.avro.file.FileReader; import org.apache.avro.file.SeekableByteArrayInput; @@ -51,6 +52,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import java.util.Collections; import java.util.HashMap; import java.util.List; @@ -164,26 +166,46 @@ public static Option serializeAvroMetadat return Option.of(baos.toByteArray()); } - public static HoodieCleanerPlan deserializeCleanerPlan(byte[] bytes) throws IOException { - return deserializeAvroMetadata(bytes, HoodieCleanerPlan.class); + public static HoodieCleanerPlan deserializeCleanerPlan(InputStream in) throws IOException { + return deserializeAvroMetadata(in, HoodieCleanerPlan.class); } public static HoodieCompactionPlan deserializeCompactionPlan(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieCompactionPlan.class); } + public static HoodieCompactionPlan deserializeCompactionPlan(InputStream in) throws IOException { + return deserializeAvroMetadata(in, HoodieCompactionPlan.class); + } + public static HoodieCleanMetadata deserializeHoodieCleanMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieCleanMetadata.class); } + public static HoodieCleanMetadata deserializeHoodieCleanMetadata(InputStream inputStream) throws IOException { + return deserializeAvroMetadata(inputStream, HoodieCleanMetadata.class); + } + public static HoodieRollbackMetadata deserializeHoodieRollbackMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieRollbackMetadata.class); } + public static HoodieRollbackMetadata deserializeHoodieRollbackMetadata(InputStream in) throws IOException { + return deserializeAvroMetadata(in, HoodieRollbackMetadata.class); + } + public static HoodieRestoreMetadata deserializeHoodieRestoreMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieRestoreMetadata.class); } + public static HoodieRestoreMetadata deserializeHoodieRestoreMetadata(InputStream in) throws IOException { + return deserializeAvroMetadata(in, HoodieRestoreMetadata.class); + } + + public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(InputStream instantStream) throws IOException { + return deserializeAvroMetadata(instantStream, HoodieSavepointMetadata.class); + } + public static HoodieSavepointMetadata deserializeHoodieSavepointMetadata(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieSavepointMetadata.class); } @@ -192,16 +214,20 @@ public static HoodieRequestedReplaceMetadata deserializeRequestedReplaceMetadata return deserializeAvroMetadata(bytes, HoodieRequestedReplaceMetadata.class); } + public static HoodieRequestedReplaceMetadata deserializeRequestedReplaceMetadata(InputStream in) throws IOException { + return deserializeAvroMetadata(in, HoodieRequestedReplaceMetadata.class); + } + public static HoodieIndexPlan deserializeIndexPlan(byte[] bytes) throws IOException { return deserializeAvroMetadata(bytes, HoodieIndexPlan.class); } - public static HoodieCommitMetadata deserializeCommitMetadata(byte[] bytes) throws IOException { - return deserializeAvroMetadata(bytes, HoodieCommitMetadata.class); + public static HoodieCommitMetadata deserializeCommitMetadata(InputStream instantStream) throws IOException { + return deserializeAvroMetadata(instantStream, HoodieCommitMetadata.class); } - public static HoodieReplaceCommitMetadata deserializeReplaceCommitMetadata(byte[] bytes) throws IOException { - return deserializeAvroMetadata(bytes, HoodieReplaceCommitMetadata.class); + public static HoodieReplaceCommitMetadata deserializeReplaceCommitMetadata(InputStream instantStream) throws IOException { + return deserializeAvroMetadata(instantStream, HoodieReplaceCommitMetadata.class); } public static T deserializeAvroMetadata(byte[] bytes, Class clazz) @@ -211,4 +237,12 @@ public static T deserializeAvroMetadata(byte[] bytes, Class clazz) ValidationUtils.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz); return fileReader.next(); } + + public static T deserializeAvroMetadata(InputStream instantStream, Class clazz) + throws IOException { + DatumReader reader = new SpecificDatumReader<>(clazz); + DataFileStream fileReader = new DataFileStream<>(instantStream, reader); + ValidationUtils.checkArgument(fileReader.hasNext(), "Could not deserialize metadata of type " + clazz); + return fileReader.next(); + } } 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 5992f06ede012..b4258470f4a5f 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 @@ -43,6 +43,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.InputStream; import java.text.ParseException; import java.util.AbstractMap; import java.util.Collection; @@ -108,8 +109,8 @@ public static List getDroppedPartitions(HoodieTableMetaClient metaClient Map partitionToLatestDeleteTimestamp = replaceCommitTimeline.getInstantsAsStream() .map(instant -> { try { - HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( - replaceCommitTimeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + instant, replaceCommitTimeline.getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); return Pair.of(instant, commitMetadata); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions modified at " + instant, e); @@ -124,7 +125,7 @@ public static List getDroppedPartitions(HoodieTableMetaClient metaClient cleanerTimeline.getInstantsAsStream() .forEach(instant -> { try { - HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(cleanerTimeline.getInstantDetails(instant).get()); + HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(cleanerTimeline.getInstantContentStream(instant)); cleanMetadata.getPartitionMetadata().forEach((partition, partitionMetadata) -> { if (Boolean.TRUE.equals(partitionMetadata.getIsPartitionDeleted())) { partitionToLatestDeleteTimestamp.put(partition, instant.requestedTime()); @@ -170,15 +171,16 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { case DELTA_COMMIT_ACTION: try { CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); - HoodieCommitMetadata commitMetadata = metadataSerDe.deserialize(s, timeline.getInstantDetails(s).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metadataSerDe.deserialize(s, timeline.getInstantContentStream(s), HoodieCommitMetadata.class); return commitMetadata.getPartitionToWriteStats().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions written at " + s, e); } case REPLACE_COMMIT_ACTION: try { - HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( - timeline.getInstantDetails(s).get(), HoodieReplaceCommitMetadata.class); + CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); + HoodieReplaceCommitMetadata commitMetadata = metadataSerDe.deserialize( + s, timeline.getInstantContentStream(s), HoodieReplaceCommitMetadata.class); Set partitions = new HashSet<>(); partitions.addAll(commitMetadata.getPartitionToReplaceFileIds().keySet()); partitions.addAll(commitMetadata.getPartitionToWriteStats().keySet()); @@ -188,20 +190,20 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { } case HoodieTimeline.CLEAN_ACTION: try { - HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantDetails(s).get()); + HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(timeline.getInstantContentStream(s)); return cleanMetadata.getPartitionMetadata().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions cleaned at " + s, e); } case HoodieTimeline.ROLLBACK_ACTION: try { - return TimelineMetadataUtils.deserializeHoodieRollbackMetadata(timeline.getInstantDetails(s).get()).getPartitionMetadata().keySet().stream(); + return TimelineMetadataUtils.deserializeHoodieRollbackMetadata(timeline.getInstantContentStream(s)).getPartitionMetadata().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions rolledback at " + s, e); } case HoodieTimeline.RESTORE_ACTION: try { - HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantDetails(s).get(), HoodieRestoreMetadata.class); + HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantContentStream(s), HoodieRestoreMetadata.class); return restoreMetadata.getHoodieRestoreMetadata().values().stream() .flatMap(Collection::stream) .flatMap(rollbackMetadata -> rollbackMetadata.getPartitionMetadata().keySet().stream()); @@ -210,7 +212,7 @@ public static List getAffectedPartitions(HoodieTimeline timeline) { } case HoodieTimeline.SAVEPOINT_ACTION: try { - return TimelineMetadataUtils.deserializeHoodieSavepointMetadata(timeline.getInstantDetails(s).get()).getPartitionMetadata().keySet().stream(); + return TimelineMetadataUtils.deserializeHoodieSavepointMetadata(timeline.getInstantContentStream(s)).getPartitionMetadata().keySet().stream(); } catch (IOException e) { throw new HoodieIOException("Failed to get partitions savepoint at " + s, e); } @@ -257,7 +259,7 @@ private static Option getMetadataValue(HoodieTableMetaClient metaClient, try { LOG.info("reading checkpoint info for:" + instant + " key: " + extraMetadataKey); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - metaClient.getCommitsTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getCommitsTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); return Option.ofNullable(commitMetadata.getExtraMetadata().get(extraMetadataKey)); } catch (IOException e) { @@ -271,8 +273,11 @@ public static boolean isClusteringCommit(HoodieTableMetaClient metaClient, Hoodi if (REPLACE_COMMIT_ACTION.equals(completedInstant.getAction())) { // replacecommit is used for multiple operations: insert_overwrite/cluster etc. // Check operation type to see if this instant is related to clustering. - HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes( - metaClient.getActiveTimeline().getInstantDetails(completedInstant).get(), HoodieReplaceCommitMetadata.class); + + HoodieReplaceCommitMetadata replaceMetadata = metaClient.getCommitMetadataSerDe().deserialize( + completedInstant, + metaClient.getActiveTimeline().getInstantContentStream(completedInstant), + HoodieReplaceCommitMetadata.class); return WriteOperationType.CLUSTER.equals(replaceMetadata.getOperationType()); } @@ -334,12 +339,13 @@ public static HoodieTimeline getCommitsTimelineAfter( public static HoodieCommitMetadata getCommitMetadata( HoodieInstant instant, HoodieTimeline timeline) throws IOException { - byte[] data = timeline.getInstantDetails(instant).get(); + InputStream inputStream = timeline.getInstantContentStream(instant); if (instant.getAction().equals(REPLACE_COMMIT_ACTION) || instant.getAction().equals(CLUSTERING_ACTION)) { - return HoodieReplaceCommitMetadata.fromBytes(data, HoodieReplaceCommitMetadata.class); + CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); + return metadataSerDe.deserialize(instant, inputStream, HoodieReplaceCommitMetadata.class); } else { CommitMetadataSerDe metadataSerDe = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()).getCommitMetadataSerDe(); - return metadataSerDe.deserialize(instant, data, HoodieCommitMetadata.class); + return metadataSerDe.deserialize(instant, inputStream, HoodieCommitMetadata.class); } } 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 index d832d7f4baa46..218bc80186e46 100644 --- 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 @@ -288,7 +288,7 @@ private Stream> getCommitMetadataStrea .map(instant -> { try { HoodieCommitMetadata commitMetadata = - metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantContentStream(instant), HoodieCommitMetadata.class); return Pair.of(instant, commitMetadata); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); 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 index 48567a64ffd0e..fd055e14b60ac 100644 --- 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 @@ -25,20 +25,24 @@ import org.apache.hudi.common.util.Option; import java.io.IOException; +import java.io.InputStream; -import static org.apache.hudi.common.util.StringUtils.fromUTF8Bytes; +import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.isEmptyStream; public class CommitMetadataSerDeV1 implements CommitMetadataSerDe { @Override - public T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException { + public T deserialize(HoodieInstant instant, InputStream inputStream, Class clazz) throws IOException { try { - if (bytes.length == 0) { + if (isEmptyStream(inputStream)) { return clazz.newInstance(); } - return fromJsonString(fromUTF8Bytes(bytes), clazz); + + // Use ObjectMapper to directly read from InputStream + // This avoids loading entire content into memory at once + return JsonUtils.getObjectMapper().readValue(inputStream, clazz); } catch (Exception e) { - throw new IOException("unable to read commit metadata for instant " + instant + " bytes length: " + bytes.length, e); + throw new IOException("Unable to read commit metadata for instant " + instant, e); } } 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 index e23fe191cb568..0956c6b4a1df5 100644 --- 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 @@ -303,7 +303,7 @@ private Stream> getCommitMetadataStrea .map(instant -> { try { HoodieCommitMetadata commitMetadata = - metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantDetails(instant).get(), HoodieCommitMetadata.class); + metaClient.getCommitMetadataSerDe().deserialize(instant, getInstantContentStream(instant), HoodieCommitMetadata.class); return Pair.of(instant, commitMetadata); } catch (IOException e) { throw new HoodieIOException(String.format("Failed to fetch HoodieCommitMetadata for instant (%s)", instant), e); 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 index 7b95bc6f3382b..64257218e704b 100644 --- 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 @@ -34,44 +34,43 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.io.InputStream; import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertCommitMetadataAvroToPojo; import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.convertReplaceCommitMetadataAvroToPojo; +import static org.apache.hudi.common.table.timeline.MetadataConversionUtils.isEmptyStream; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeCommitMetadata; import static org.apache.hudi.common.table.timeline.TimelineMetadataUtils.deserializeReplaceCommitMetadata; public class CommitMetadataSerDeV2 implements CommitMetadataSerDe { @Override - public T deserialize(HoodieInstant instant, byte[] bytes, Class clazz) throws IOException { + public T deserialize(HoodieInstant instant, InputStream inputStream, Class clazz) throws IOException { try { - if (bytes.length == 0) { + if (isEmptyStream(inputStream)) { return clazz.newInstance(); } + if (instant.isLegacy()) { // For legacy instant, delegate to legacy SerDe. try { - return new CommitMetadataSerDeV1().deserialize(instant, bytes, clazz); + return new CommitMetadataSerDeV1().deserialize(instant, inputStream, clazz); } catch (Exception e) { throw new IOException("unable to read legacy commit metadata for instant " + instant, e); } } // For any new commit metadata class being added, we need the corresponding logic added here if (org.apache.hudi.common.model.HoodieReplaceCommitMetadata.class.isAssignableFrom(clazz)) { - return (T) convertReplaceCommitMetadataAvroToPojo(deserializeReplaceCommitMetadata(bytes)); + return (T) convertReplaceCommitMetadataAvroToPojo(deserializeReplaceCommitMetadata(inputStream)); } - return (T) convertCommitMetadataAvroToPojo(deserializeCommitMetadata(bytes)); + return (T) convertCommitMetadataAvroToPojo(deserializeCommitMetadata(inputStream)); } catch (Exception e) { - throw new IOException("unable to read commit metadata for instant " + instant + " bytes length: " + bytes.length, e); + throw new IOException("unable to read commit metadata for instant " + instant, 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); + public static T fromJsonString(InputStream inputStream, Class clazz) throws Exception { + return JsonUtils.getObjectMapper().readValue(inputStream, clazz); } @Override 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 bcc32e0106480..58d496f53d3fd 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 @@ -263,7 +263,9 @@ private void resetFileGroupsReplaced(HoodieTimeline timeline) { HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline(); Stream> resultStream = replacedTimeline.getInstantsAsStream().flatMap(instant -> { try { - HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieReplaceCommitMetadata replaceMetadata = metaClient.getCommitMetadataSerDe().deserialize( + instant, + metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); // get replace instant mapping for each partition, fileId 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 078dc38ae1622..d18919490cf24 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 @@ -37,6 +37,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineDiffHelper; import org.apache.hudi.common.table.timeline.TimelineDiffHelper.TimelineDiffResult; +import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.ClusteringUtils; @@ -256,8 +257,8 @@ private void addPendingLogCompactionInstant(HoodieInstant instant) throws IOExce */ private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing committed instant (" + instant + ")"); - HoodieCommitMetadata commitMetadata = - metaClient.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + instant, timeline.getInstantContentStream(instant), HoodieCommitMetadata.class); updatePartitionWriteFileGroups(commitMetadata.getPartitionToWriteStats(), timeline, instant); LOG.info("Done Syncing committed instant (" + instant + ")"); } @@ -293,7 +294,7 @@ private void updatePartitionWriteFileGroups(Map> p private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing restore instant (" + instant + ")"); HoodieRestoreMetadata metadata = - TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRestoreMetadata.class); + TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantContentStream(instant), HoodieRestoreMetadata.class); Map>> partitionFiles = metadata.getHoodieRestoreMetadata().entrySet().stream().flatMap(entry -> { @@ -324,7 +325,7 @@ private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) t private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing rollback instant (" + instant + ")"); HoodieRollbackMetadata metadata = - TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantDetails(instant).get(), HoodieRollbackMetadata.class); + TimelineMetadataUtils.deserializeAvroMetadata(timeline.getInstantContentStream(instant), HoodieRollbackMetadata.class); metadata.getPartitionMetadata().entrySet().stream().forEach(e -> { removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().getSuccessDeleteFiles()); @@ -340,8 +341,9 @@ private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) */ private void addReplaceInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { LOG.info("Syncing replace instant (" + instant + ")"); - HoodieReplaceCommitMetadata replaceMetadata = - HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); + HoodieReplaceCommitMetadata replaceMetadata = layout.getCommitMetadataSerDe().deserialize( + instant, timeline.getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); updatePartitionWriteFileGroups(replaceMetadata.getPartitionToWriteStats(), timeline, instant); replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().forEach(entry -> { String partition = entry.getKey(); 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 4ac802e2314c8..3cd9d1904d2ff 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 @@ -41,6 +41,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.io.InputStream; import java.time.Instant; import java.time.ZonedDateTime; import java.util.Date; @@ -109,7 +110,7 @@ public static HoodieCleanMetadata getCleanerMetadata(HoodieTableMetaClient metaC throws IOException { CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient); HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata( - metaClient.getActiveTimeline().readCleanerInfoAsBytes(cleanInstant).get()); + metaClient.getActiveTimeline().getInstantContentStream(cleanInstant)); return metadataMigrator.upgradeToLatest(cleanMetadata, cleanMetadata.getVersion()); } @@ -125,6 +126,13 @@ public static HoodieCleanMetadata getCleanerMetadata(HoodieTableMetaClient metaC return metadataMigrator.upgradeToLatest(cleanMetadata, cleanMetadata.getVersion()); } + public static HoodieCleanMetadata getCleanerMetadata(HoodieTableMetaClient metaClient, InputStream in) + throws IOException { + CleanMetadataMigrator metadataMigrator = new CleanMetadataMigrator(metaClient); + HoodieCleanMetadata cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata(in); + return metadataMigrator.upgradeToLatest(cleanMetadata, cleanMetadata.getVersion()); + } + public static Option getEarliestCommitToRetain( HoodieTimeline commitsTimeline, HoodieCleaningPolicy cleaningPolicy, int commitsRetained, Instant latestInstant, int hoursRetained, HoodieTimelineTimeZone timeZone) { @@ -170,7 +178,7 @@ public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, throws IOException { CleanPlanMigrator cleanPlanMigrator = new CleanPlanMigrator(metaClient); HoodieCleanerPlan cleanerPlan = TimelineMetadataUtils.deserializeAvroMetadata( - metaClient.getActiveTimeline().readCleanerInfoAsBytes(cleanInstant).get(), HoodieCleanerPlan.class); + metaClient.getActiveTimeline().getInstantContentStream(cleanInstant), HoodieCleanerPlan.class); return cleanPlanMigrator.upgradeToLatest(cleanerPlan, cleanerPlan.getVersion()); } @@ -188,6 +196,13 @@ public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, return cleanPlanMigrator.upgradeToLatest(cleanerPlan, cleanerPlan.getVersion()); } + public static HoodieCleanerPlan getCleanerPlan(HoodieTableMetaClient metaClient, InputStream in) + throws IOException { + CleanPlanMigrator cleanPlanMigrator = new CleanPlanMigrator(metaClient); + HoodieCleanerPlan cleanerPlan = TimelineMetadataUtils.deserializeAvroMetadata(in, HoodieCleanerPlan.class); + return cleanPlanMigrator.upgradeToLatest(cleanerPlan, cleanerPlan.getVersion()); + } + /** * Convert list of cleanFileInfo instances to list of avro-generated HoodieCleanFileInfo instances. * @param cleanFileInfoList 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 988a620c6d28d..c0e5baafd3581 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 @@ -129,7 +129,7 @@ public static Option buildMetadataFromInstant(HoodieTimeli try { TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize(instant, - timeline.getInstantDetails(instant).get(), + timeline.getInstantContentStream(instant), HoodieCommitMetadata.class); return Option.of(commitMetadata); @@ -186,7 +186,7 @@ public static Option getValidCheckpointForCurrentWriter(HoodieTimeline t .map(instant -> { try { HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() - .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, timeline.getInstantContentStream(instant), 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 4a77fcbf9d516..b813eb8223837 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 @@ -36,6 +36,7 @@ import org.apache.hudi.exception.HoodieException; import java.io.IOException; +import java.io.InputStream; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -202,6 +203,16 @@ public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaC } } + public static HoodieCompactionPlan getCompactionPlan(HoodieTableMetaClient metaClient, InputStream in) { + CompactionPlanMigrator migrator = new CompactionPlanMigrator(metaClient); + try { + HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeCompactionPlan(in); + return migrator.upgradeToLatest(compactionPlan, compactionPlan.getVersion()); + } catch (IOException e) { + throw new HoodieException(e); + } + } + /** * Get all PartitionPath + file-ids with pending Compaction operations and their target compaction instant time. * 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 51f9587433c27..7c577bd8f1f98 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 @@ -18,6 +18,9 @@ package org.apache.hudi.common.util; +import com.github.benmanes.caffeine.cache.Cache; +import com.github.benmanes.caffeine.cache.Caffeine; +import org.apache.avro.Schema; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -36,10 +39,6 @@ import org.apache.hudi.internal.schema.utils.SerDeHelper; import org.apache.hudi.storage.HoodieStorage; import org.apache.hudi.storage.StoragePath; - -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; @@ -79,7 +78,7 @@ public class InternalSchemaCache { * first step: try to get internalSchema from hoodie commit files, we no need to add lock. * if we cannot get internalSchema by first step, then we try to get internalSchema from cache. * - * @param versionID schema version_id need to search + * @param versionID schema version_id need to search * @param metaClient current hoodie metaClient * @return internalSchema */ @@ -123,8 +122,8 @@ private static Option getSchemaByReadingCommitFile(long versionI if (instants.isEmpty()) { return Option.empty(); } - byte[] data = timeline.getInstantDetails(instants.get(0)).get(); - HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instants.get(0), data, HoodieCommitMetadata.class); + HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize( + instants.get(0), timeline.getInstantContentStream(instants.get(0)), HoodieCommitMetadata.class); String latestInternalSchemaStr = metadata.getMetadata(SerDeHelper.LATEST_SCHEMA); return SerDeHelper.fromJson(latestInternalSchemaStr); } catch (Exception e) { @@ -135,20 +134,22 @@ private static Option getSchemaByReadingCommitFile(long versionI /** * Get internalSchema and avroSchema for compaction/cluster operation. * - * @param metaClient current hoodie metaClient + * @param metaClient current hoodie metaClient * @param compactionAndClusteringInstant first instant before current compaction/cluster instant * @return (internalSchemaStrOpt, avroSchemaStrOpt) a pair of InternalSchema/avroSchema */ public static Pair, Option> getInternalSchemaAndAvroSchemaForClusteringAndCompaction(HoodieTableMetaClient metaClient, String compactionAndClusteringInstant) { // try to load internalSchema to support Schema Evolution HoodieTimeline timelineBeforeCurrentCompaction = metaClient.getCommitsAndCompactionTimeline().findInstantsBefore(compactionAndClusteringInstant).filterCompletedInstants(); - Option lastInstantBeforeCurrentCompaction = timelineBeforeCurrentCompaction.lastInstant(); + Option lastInstantBeforeCurrentCompaction = timelineBeforeCurrentCompaction.lastInstant(); if (lastInstantBeforeCurrentCompaction.isPresent()) { // try to find internalSchema - byte[] data = timelineBeforeCurrentCompaction.getInstantDetails(lastInstantBeforeCurrentCompaction.get()).get(); HoodieCommitMetadata metadata; try { - metadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstantBeforeCurrentCompaction.get(), data, HoodieCommitMetadata.class); + metadata = metaClient.getCommitMetadataSerDe().deserialize( + lastInstantBeforeCurrentCompaction.get(), + timelineBeforeCurrentCompaction.getInstantContentStream(lastInstantBeforeCurrentCompaction.get()), + HoodieCommitMetadata.class); } catch (Exception e) { throw new HoodieException(String.format("cannot read metadata from commit: %s", lastInstantBeforeCurrentCompaction.get()), e); } @@ -174,13 +175,13 @@ public static Pair, Option> getInternalSchemaAndAvroSchem * if we cannot parser internalSchema in step2 (eg: schema evolution is not enabled when we create hoodie table, however after some inserts we enable schema evolution) * try to convert table schema to internalSchema. * - * @param versionId the internalSchema version to be search. - * @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 versionId the internalSchema version to be search. + * @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 + * @param instantGenerator InstantGenerator * @return a internalSchema. */ public static InternalSchema getInternalSchemaByVersionId(long versionId, String tablePath, HoodieStorage storage, String validCommits, @@ -197,15 +198,14 @@ public static InternalSchema getInternalSchemaByVersionId(long versionId, String .findFirst().map(f -> new StoragePath(hoodieMetaPath, f)).orElse(null); if (candidateCommitFile != null) { try { - byte[] data; + HoodieCommitMetadata metadata; try (InputStream is = storage.open(candidateCommitFile)) { - data = FileIOUtils.readAsByteArray(is); + metadata = commitMetadataSerDe.deserialize(instantGenerator.createNewInstant( + new StoragePathInfo(candidateCommitFile, -1, false, (short) 0, 0L, 0L)), + is, HoodieCommitMetadata.class); } catch (IOException e) { throw e; } - 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) { 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 53fe63c9625b6..2a4d639adfb33 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 @@ -279,7 +279,7 @@ public static boolean hasCommitConflict(HoodieActiveTimeline activeTimeline, Set Set missingFileIDs = currentInstants.stream().flatMap(instant -> { try { TimelineLayout layout = TimelineLayout.fromVersion(activeTimeline.getTimelineLayoutVersion()); - return layout.getCommitMetadataSerDe().deserialize(instant, activeTimeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class) + return layout.getCommitMetadataSerDe().deserialize(instant, activeTimeline.getInstantContentStream(instant), HoodieCommitMetadata.class) .getFileIdAndRelativePaths().keySet().stream(); } catch (Exception e) { return Stream.empty(); 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 bcc73c737c6cb..e5af2fef1dbc3 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 @@ -1093,7 +1093,7 @@ private static void reAddLogFilesFromRollbackPlan(HoodieTableMetaClient dataTabl HoodieInstant requested = factory.getRollbackRequestedInstant(rollbackInstant); try { HoodieRollbackPlan rollbackPlan = TimelineMetadataUtils.deserializeAvroMetadata( - dataTableMetaClient.getActiveTimeline().readRollbackInfoAsBytes(requested).get(), HoodieRollbackPlan.class); + dataTableMetaClient.getActiveTimeline().getInstantContentStream(requested), HoodieRollbackPlan.class); rollbackPlan.getRollbackRequests().forEach(rollbackRequest -> { final String partitionId = getPartitionIdentifierForFilesPartition(rollbackRequest.getPartitionPath()); @@ -2012,13 +2012,13 @@ private static List getRollbackedCommits(HoodieInstant instant, HoodieAc if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { try { HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.deserializeHoodieRollbackMetadata( - timeline.getInstantDetails(instant).get()); + timeline.getInstantContentStream(instant)); commitsToRollback = rollbackMetadata.getCommitsRollback(); } catch (IOException e) { // if file is empty, fetch the commits to rollback from rollback.requested file HoodieRollbackPlan rollbackPlan = TimelineMetadataUtils.deserializeAvroMetadata( - timeline.readRollbackInfoAsBytes(factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, - instant.requestedTime())).get(), HoodieRollbackPlan.class); + timeline.getInstantContentStream(factory.createNewInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, + instant.requestedTime())), HoodieRollbackPlan.class); commitsToRollback = Collections.singletonList(rollbackPlan.getInstantToRollback().getCommitTime()); LOG.warn("Had to fetch rollback info from requested instant since completed file is empty {}", instant); } 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 6ced2d675a51d..5a011512aad87 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 @@ -26,11 +26,11 @@ import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.JsonUtils; - import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.junit.jupiter.api.Test; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.util.Arrays; import java.util.Collections; @@ -121,15 +121,15 @@ public void testGetFileSliceForFileGroupFromDeltaCommit() throws IOException { org.apache.hudi.avro.model.HoodieWriteStat writeStat1 = createWriteStat("111", "111base", Arrays.asList("1.log", "2.log")); org.apache.hudi.avro.model.HoodieWriteStat writeStat2 = createWriteStat("111", "111base", Arrays.asList("3.log", "4.log")); org.apache.hudi.avro.model.HoodieWriteStat writeStat3 = createWriteStat("222", null, Collections.singletonList("5.log")); - Map> partitionToWriteStatsMap = new HashMap<>(); + Map> partitionToWriteStatsMap = new HashMap<>(); partitionToWriteStatsMap.put("partition1", Arrays.asList(writeStat2, writeStat3)); partitionToWriteStatsMap.put("partition2", Collections.singletonList(writeStat1)); commitMetadata.setPartitionToWriteStats(partitionToWriteStatsMap); byte[] serializedCommitMetadata = TimelineMetadataUtils.serializeAvroMetadata( - commitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class).get(); + commitMetadata, org.apache.hudi.avro.model.HoodieCommitMetadata.class).get(); Option>> result = HoodieCommitMetadata.getFileSliceForFileGroupFromDeltaCommit( - serializedCommitMetadata, new HoodieFileGroupId("partition1","111")); + new ByteArrayInputStream(serializedCommitMetadata), new HoodieFileGroupId("partition1", "111")); assertTrue(result.isPresent()); assertEquals("111base", result.get().getKey()); @@ -138,7 +138,7 @@ public void testGetFileSliceForFileGroupFromDeltaCommit() throws IOException { assertEquals("4.log", result.get().getValue().get(1)); result = HoodieCommitMetadata.getFileSliceForFileGroupFromDeltaCommit( - serializedCommitMetadata, new HoodieFileGroupId("partition1","222")); + new ByteArrayInputStream(serializedCommitMetadata), new HoodieFileGroupId("partition1", "222")); assertTrue(result.isPresent()); assertTrue(result.get().getKey().isEmpty()); assertEquals(1, result.get().getValue().size()); @@ -151,7 +151,7 @@ public void testCommitMetadataSerde() throws Exception { 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<>(); + Map> partitionToWriteStatsMap = new HashMap<>(); partitionToWriteStatsMap.put("partition1", Arrays.asList(writeStat1, writeStat2)); partitionToWriteStatsMap.put("partition2", Collections.singletonList(writeStat3)); commitMetadata.setPartitionToWriteStats(partitionToWriteStatsMap); @@ -161,7 +161,7 @@ public void testCommitMetadataSerde() throws Exception { 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); + new ByteArrayInputStream(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()); @@ -170,12 +170,12 @@ public void testCommitMetadataSerde() throws Exception { 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); + 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); + COMMIT_METADATA_SER_DE.deserialize(legacyInstant, new ByteArrayInputStream(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()); @@ -190,6 +190,6 @@ private org.apache.hudi.avro.model.HoodieWriteStat createWriteStat(String fileId writeStat.setFileId(fileId); writeStat.setBaseFile(baseFile); writeStat.setLogFiles(logFiles); - return writeStat; + return writeStat; } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v1/TestCommitMetadataSerDeV1.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v1/TestCommitMetadataSerDeV1.java index f354ab20b96f2..e19cf9f0cdd62 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v1/TestCommitMetadataSerDeV1.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v1/TestCommitMetadataSerDeV1.java @@ -29,6 +29,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import java.io.ByteArrayInputStream; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -62,10 +63,10 @@ public void setUp() { public void testEmptyMetadataSerDe() throws Exception { // Create empty commit metadata HoodieCommitMetadata emptyMetadata = new HoodieCommitMetadata(); - + // Create SerDe instance CommitMetadataSerDeV1 serDe = new CommitMetadataSerDeV1(); - + // Create test instant HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, "commit", "001"); @@ -74,7 +75,7 @@ public void testEmptyMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieCommitMetadata.class); // Verify assertNotNull(deserialized); @@ -134,7 +135,7 @@ public void testPopulatedMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieCommitMetadata.class); // Verify all fields assertNotNull(deserialized); @@ -212,7 +213,7 @@ public void testReplaceCommitMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieReplaceCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieReplaceCommitMetadata.class); // Verify basic fields assertNotNull(deserialized); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v2/TestCommitMetadataSerDeV2.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v2/TestCommitMetadataSerDeV2.java index a849763fb83ef..1e0167a0d9746 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v2/TestCommitMetadataSerDeV2.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/versioning/v2/TestCommitMetadataSerDeV2.java @@ -29,6 +29,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; +import java.io.ByteArrayInputStream; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -74,7 +75,7 @@ public void testEmptyMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieCommitMetadata.class); // Verify assertNotNull(deserialized); @@ -136,7 +137,7 @@ public void testPopulatedMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieCommitMetadata.class); + HoodieCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieCommitMetadata.class); // Verify all fields assertNotNull(deserialized); @@ -226,7 +227,7 @@ public void testReplaceCommitMetadataSerDe() throws Exception { assertTrue(serialized.isPresent()); // Deserialize - HoodieReplaceCommitMetadata deserialized = serDe.deserialize(instant, serialized.get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata deserialized = serDe.deserialize(instant, new ByteArrayInputStream(serialized.get()), HoodieReplaceCommitMetadata.class); // Verify basic fields assertNotNull(deserialized); 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 125a2161e5bdb..68fb88a6c4b64 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 @@ -28,6 +28,7 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; 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.StringUtils; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.configuration.FlinkOptions; @@ -138,9 +139,10 @@ private ConsistentBucketIdentifier getBucketIdentifier(String partition) { public void snapshotState(FunctionSnapshotContext functionSnapshotContext) throws Exception { HoodieTimeline timeline = writeClient.getHoodieTable().getActiveTimeline().getCompletedReplaceTimeline().findInstantsAfter(lastRefreshInstant); if (!timeline.empty()) { + TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); for (HoodieInstant instant : timeline.getInstants()) { - HoodieReplaceCommitMetadata commitMetadata = HoodieReplaceCommitMetadata.fromBytes( - timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata commitMetadata = layout.getCommitMetadataSerDe().deserialize( + instant, timeline.getInstantContentStream(instant), HoodieReplaceCommitMetadata.class); Set affectedPartitions = commitMetadata.getPartitionToReplaceFileIds().keySet(); 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); 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 c11d54239000e..4e8d25e260f16 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 @@ -173,9 +173,8 @@ public static Option getCommitMetadataSafely( HoodieInstant instant, HoodieTimeline timeline) { try { - byte[] data = timeline.getInstantDetails(instant).get(); TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); - return Option.of(layout.getCommitMetadataSerDe().deserialize(instant, data, HoodieCommitMetadata.class)); + return Option.of(layout.getCommitMetadataSerDe().deserialize(instant, timeline.getInstantContentStream(instant), HoodieCommitMetadata.class)); } catch (FileNotFoundException fe) { // make this fail safe. LOG.warn("Instant {} was deleted by the cleaner, ignore", instant.requestedTime()); diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriterWithPartitionTTl.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriterWithPartitionTTl.java index 9bedb32f13940..57dfaf6800eea 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriterWithPartitionTTl.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/TestWriterWithPartitionTTl.java @@ -84,7 +84,7 @@ public void testFlinkWriterWithPartitionTTL() throws Exception { HoodieActiveTimeline timeline = StreamerUtil.createMetaClient(conf).getActiveTimeline(); assertTrue(timeline.getCompletedReplaceTimeline().getInstants().size() > 0); HoodieInstant replaceCommit = timeline.getCompletedReplaceTimeline().getInstants().get(0); - HoodieReplaceCommitMetadata commitMetadata = TimelineMetadataUtils.deserializeReplaceCommitMetadata(timeline.getInstantDetails(replaceCommit).get()); + HoodieReplaceCommitMetadata commitMetadata = TimelineMetadataUtils.deserializeReplaceCommitMetadata(timeline.getInstantContentStream(replaceCommit)); assertTrue(commitMetadata.getPartitionToReplaceFileIds().containsKey("par1")); } } 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 23c4f0b03243c..a0aed3bbdee9d 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 @@ -105,7 +105,7 @@ private static void doDeleteCommit(String tablePath, boolean isCow) throws Excep String filename = INSTANT_FILE_NAME_GENERATOR.getFileName(activeCompletedTimeline.getInstants().get(0)); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), + .deserialize(instant, metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); // delete successful commit to simulate an unsuccessful write 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 d03d004e1d35d..875650683da37 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 @@ -18,7 +18,10 @@ package org.apache.hudi.common.table.timeline; +import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.common.fs.NoOpConsistencyGuard; +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.HoodieInstant.State; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; @@ -38,6 +41,7 @@ import org.junit.jupiter.api.Test; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.text.ParseException; import java.util.ArrayList; import java.util.Collections; @@ -64,6 +68,8 @@ import static org.apache.hudi.common.testutils.HoodieTestUtils.INSTANT_FILE_NAME_PARSER; 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.getCleanerMetadata; +import static org.apache.hudi.common.util.CleanerUtils.getCleanerPlan; import static org.apache.hudi.common.util.StringUtils.getUTF8Bytes; import static org.hamcrest.CoreMatchers.is; import static org.hamcrest.MatcherAssert.assertThat; @@ -625,11 +631,51 @@ public void testRollbackActionsTimeline() { assertEquals(0, timeline.filterRequestedRollbackTimeline().countInstants()); } + @Test + void testParsingCommitDetails() throws IOException { + HoodieInstant commitInstant = metaClient.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); + HoodieInstant cleanInstant = metaClient.createNewInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, "2"); + + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + HoodieWriteStat hoodieWriteStat = new HoodieWriteStat(); + hoodieWriteStat.setFileId("file_id1"); + hoodieWriteStat.setPath("path1"); + hoodieWriteStat.setPrevCommit("1"); + commitMetadata.addWriteStat("partition1", hoodieWriteStat); + timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); + timeline.createNewInstant(commitInstant); + timeline.transitionRequestedToInflight(commitInstant, Option.empty()); + HoodieInstant completeCommitInstant = metaClient.createNewInstant(State.INFLIGHT, commitInstant.getAction(), commitInstant.requestedTime()); + timeline.saveAsComplete(completeCommitInstant, + Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + HoodieActiveTimeline timelineAfterFirstInstant = timeline.reload(); + + HoodieInstant completedCommitInstant = timelineAfterFirstInstant.lastInstant().get(); + assertEquals(commitMetadata, metaClient.getCommitMetadataSerDe().deserialize( + completedCommitInstant, timelineAfterFirstInstant.getInstantContentStream(completedCommitInstant), HoodieCommitMetadata.class)); + + HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(); + cleanerPlan.setLastCompletedCommitTimestamp("1"); + cleanerPlan.setPolicy("policy"); + cleanerPlan.setVersion(1); + cleanerPlan.setPartitionsToBeDeleted(Collections.singletonList("partition1")); + timeline.saveToCleanRequested(cleanInstant, TimelineMetadataUtils.serializeCleanerPlan(cleanerPlan)); + + assertEquals(cleanerPlan, getCleanerMetadata(metaClient, cleanInstant)); + + HoodieTimeline mergedTimeline = timelineAfterFirstInstant.mergeTimeline(timeline.reload()); + assertEquals(commitMetadata, metaClient.getCommitMetadataSerDe().deserialize( + completedCommitInstant, mergedTimeline.getInstantContentStream(completedCommitInstant), HoodieCommitMetadata.class)); + assertEquals(cleanerPlan,getCleanerPlan(metaClient, cleanInstant)); + assertEquals(commitMetadata, metaClient.getCommitMetadataSerDe().deserialize( + completedCommitInstant, mergedTimeline.getInstantContentStream(completedCommitInstant), HoodieCommitMetadata.class)); + } + @Test void missingInstantCausesError() { timeline = TIMELINE_FACTORY.createActiveTimeline(metaClient); HoodieInstant commitInstant = metaClient.createNewInstant(State.REQUESTED, HoodieTimeline.COMMIT_ACTION, "1"); - assertThrows(HoodieIOException.class, () -> timeline.getInstantDetails(commitInstant)); + assertThrows(HoodieIOException.class, () -> timeline.getInstantContentStream(commitInstant)); } @Test 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 4e2039d373f26..b0cf406d17c57 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 @@ -1271,14 +1271,14 @@ private Option getCommitMeta(HoodieInstant hoodieInstant) switch (hoodieInstant.getAction()) { case REPLACE_COMMIT_ACTION: case CLUSTERING_ACTION: - HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata replaceCommitMetadata = metaClient.getCommitMetadataSerDe().deserialize( + hoodieInstant, metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieReplaceCommitMetadata.class); return Option.of(replaceCommitMetadata); case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.COMMIT_ACTION: HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( hoodieInstant, - metaClient.getActiveTimeline().getInstantDetails(hoodieInstant).get(), HoodieCommitMetadata.class); + metaClient.getActiveTimeline().getInstantContentStream(hoodieInstant), HoodieCommitMetadata.class); return Option.of(commitMetadata); default: throw new IllegalArgumentException("Unknown instant action" + hoodieInstant.getAction()); 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 aa24bd36fd92b..f130cc2c0a8c4 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 @@ -252,7 +252,7 @@ public static Option getAffectedPartitions(List commitsTo Set partitionsToList = new HashSet<>(); for (HoodieInstant commit : commitsToCheck) { HoodieCommitMetadata commitMetadata = tableMetaClient.getCommitMetadataSerDe().deserialize(commit, - timeline.getInstantDetails(commit).get(), HoodieCommitMetadata.class); + timeline.getInstantContentStream(commit), HoodieCommitMetadata.class); partitionsToList.addAll(commitMetadata.getPartitionToWriteStats().keySet()); } if (partitionsToList.isEmpty()) { 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 a3c69b228a7e2..508389be037a9 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 @@ -156,7 +156,7 @@ int getSchemaVersionFromCommit(int nthCommit) throws Exception { HoodieInstant prevInstant = timeline.nthFromLastInstant(nthCommit).get(); HoodieCommitMetadata commit = metaClient.getCommitMetadataSerDe().deserialize( prevInstant, - timeline.getInstantDetails(prevInstant).get(), + timeline.getInstantContentStream(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 70ac8c7f6c10d..257c7669568a8 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 @@ -202,7 +202,7 @@ private Option getLatestCheckpoint(HoodieTimeline timeline) { try { TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() - .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, timeline.getInstantContentStream(instant), 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/ScheduleCompactNode.java b/hudi-integ-test/src/main/java/org/apache/hudi/integ/testsuite/dag/nodes/ScheduleCompactNode.java index d32e911172c58..1750465769275 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 @@ -49,7 +49,7 @@ public void execute(ExecutionContext executionContext, int curItrCount) throws E Option lastInstant = metaClient.getActiveTimeline().getCommitsTimeline().lastInstant(); if (lastInstant.isPresent()) { HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize( - lastInstant.get(), metaClient.getActiveTimeline().getInstantDetails(lastInstant.get()).get(), HoodieCommitMetadata.class); + lastInstant.get(), metaClient.getActiveTimeline().getInstantContentStream(lastInstant.get()).get(), HoodieCommitMetadata.class); Option scheduledInstant = executionContext.getHoodieTestSuiteWriter().scheduleCompaction(Option.of(metadata .getExtraMetadata())); if (scheduledInstant.isPresent()) { 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 be7d866df92d9..c5ad808383787 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 @@ -207,8 +207,8 @@ public static Option getCommitMetadataForLatestInstant(Hoo Option latestInstant = timeline.lastInstant(); if (latestInstant.isPresent()) { try { - byte[] data = timeline.getInstantDetails(latestInstant.get()).get(); - return Option.of(metaClient.getCommitMetadataSerDe().deserialize(latestInstant.get(), data, HoodieCommitMetadata.class)); + return Option.of(metaClient.getCommitMetadataSerDe().deserialize( + latestInstant.get(), timeline.getInstantContentStream(latestInstant.get()), HoodieCommitMetadata.class)); } catch (Exception e) { throw new HoodieException("Failed to read schema from commit metadata", e); } diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV1.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV1.scala index bc0869840d0db..1ff7e92a72856 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV1.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV1.scala @@ -25,8 +25,8 @@ import org.apache.hudi.client.utils.SparkInternalSchemaConverter import org.apache.hudi.common.fs.FSUtils import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieFileFormat, HoodieRecord, HoodieReplaceCommitMetadata} import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} -import org.apache.hudi.common.table.timeline.TimelineUtils.{handleHollowCommitIfNeeded, HollowCommitHandling} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} +import org.apache.hudi.common.table.timeline.TimelineUtils.{HollowCommitHandling, handleHollowCommitIfNeeded} import org.apache.hudi.common.table.timeline.TimelineUtils.HollowCommitHandling.USE_TRANSITION_TIME import org.apache.hudi.common.util.{HoodieTimer, InternalSchemaCache} import org.apache.hudi.config.HoodieWriteConfig @@ -36,7 +36,6 @@ import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.storage.{HoodieStorageUtils, StoragePath} import org.apache.hudi.table.HoodieSparkTable - import org.apache.avro.Schema import org.apache.hadoop.fs.GlobPattern import org.apache.spark.api.java.JavaSparkContext @@ -157,9 +156,10 @@ class IncrementalRelationV1(val sqlContext: SQLContext, // create Replaced file group val replacedTimeline = commitsTimelineToReturn.getCompletedReplaceTimeline + val layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion) val replacedFile = replacedTimeline.getInstants.asScala.flatMap { instant => - val replaceMetadata = HoodieReplaceCommitMetadata. - fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata]) + val replaceMetadata = layout.getCommitMetadataSerDe.deserialize( + instant, metaClient.getActiveTimeline.getInstantContentStream(instant), classOf[HoodieReplaceCommitMetadata]) replaceMetadata.getPartitionToReplaceFileIds.entrySet().asScala.flatMap { entry => entry.getValue.asScala.map { e => val fullPath = FSUtils.constructAbsolutePath(basePath, entry.getKey).toString @@ -170,7 +170,7 @@ class IncrementalRelationV1(val sqlContext: SQLContext, for (commit <- commitsToReturn) { val metadata: HoodieCommitMetadata = metaClient.getCommitMetadataSerDe.deserialize( - commit, commitTimeline.getInstantDetails(commit).get(), classOf[HoodieCommitMetadata]) + commit, commitTimeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.requestedTime) { metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).asScala.filterNot { case (k, v) => diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV2.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV2.scala index 08423c226c20f..94456122afbce 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV2.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/IncrementalRelationV2.scala @@ -25,14 +25,13 @@ import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieFileFormat, Hoo import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver} import org.apache.hudi.common.table.log.InstantRange.RangeType import org.apache.hudi.common.table.read.IncrementalQueryAnalyzer -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline} +import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.{HoodieTimer, InternalSchemaCache} import org.apache.hudi.exception.{HoodieException, HoodieIncrementalPathNotFoundException} import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.internal.schema.InternalSchema import org.apache.hudi.internal.schema.utils.SerDeHelper import org.apache.hudi.storage.{HoodieStorageUtils, StoragePath} - import org.apache.avro.Schema import org.apache.hadoop.fs.GlobPattern import org.apache.spark.rdd.RDD @@ -141,9 +140,11 @@ class IncrementalRelationV2(val sqlContext: SQLContext, // create Replaced file group val replacedInstants = commitsToReturn.filter(_.getAction.equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) + val layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion) + val replacedFile = replacedInstants.flatMap { instant => - val replaceMetadata = HoodieReplaceCommitMetadata. - fromBytes(metaClient.getActiveTimeline.getInstantDetails(instant).get, classOf[HoodieReplaceCommitMetadata]) + val replaceMetadata = layout.getCommitMetadataSerDe.deserialize( + instant, metaClient.getActiveTimeline.getInstantContentStream(instant), classOf[HoodieReplaceCommitMetadata]) replaceMetadata.getPartitionToReplaceFileIds.entrySet().asScala.flatMap { entry => entry.getValue.asScala.map { e => val fullPath = FSUtils.constructAbsolutePath(basePath, entry.getKey).toString @@ -154,7 +155,7 @@ class IncrementalRelationV2(val sqlContext: SQLContext, for (commit <- commitsToReturn) { val metadata: HoodieCommitMetadata = metaClient.getCommitMetadataSerDe.deserialize(commit, - commitTimeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + commitTimeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) if (HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS == commit.requestedTime) { metaBootstrapFileIdToFullPath ++= metadata.getFileIdAndFullPaths(basePath).asScala.filterNot { case (k, v) => 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 dd6173f2f2836..5b46f6ed33344 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 @@ -30,7 +30,6 @@ import org.apache.hudi.exception.HoodieException import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.hadoop.fs.HadoopFSUtils.convertToStoragePath import org.apache.hudi.storage.{HoodieStorage, HoodieStorageUtils, StoragePath} - import org.apache.avro.generic.GenericRecord import org.apache.avro.specific.SpecificData import org.apache.hadoop.fs.{FileStatus, Path} @@ -38,11 +37,10 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.Row import org.apache.spark.sql.types.{DataTypes, Metadata, StructField, StructType} -import java.io.File +import java.io.{File, InputStream} import java.util import java.util.Collections import java.util.function.Supplier - import scala.collection.JavaConverters._ import scala.util.control.Breaks.break 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 3a40f59952bba..2435f9cd6958a 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 @@ -116,7 +116,7 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) for (partitionWriteStat <- commitMetadata.getPartitionToWriteStats.entrySet.asScala) { for (hoodieWriteStat <- partitionWriteStat.getValue.asScala) { rows.add(Row( @@ -150,7 +150,7 @@ class ShowArchivedCommitsProcedure(includeExtraMetadata: Boolean) extends BasePr val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) rows.add(Row(commit.requestedTime, commit.getCompletionTime, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, commitMetadata.fetchTotalFilesUpdated, commitMetadata.fetchTotalPartitionsWritten, commitMetadata.fetchTotalRecordsWritten, commitMetadata.fetchTotalUpdateRecordsWritten, 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 addbf564573dc..7f83ba256b067 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 @@ -121,10 +121,12 @@ class ShowCommitExtraMetadataProcedure() extends BaseProcedure with ProcedureBui val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { - Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize( + hoodieInstant.get, + timeline.getInstantContentStream(hoodieInstant.get), classOf[HoodieReplaceCommitMetadata])) } else { - Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.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 67fef62d3d698..73dc1d3df1e34 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 @@ -105,10 +105,12 @@ class ShowCommitFilesProcedure() extends BaseProcedure with ProcedureBuilder { val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { - Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + + Option(layout.getCommitMetadataSerDe.deserialize( + hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.get), classOf[HoodieReplaceCommitMetadata])) } else { - Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.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 aa758dbdaf1c6..8e9132a698eb8 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 @@ -119,11 +119,12 @@ class ShowCommitPartitionsProcedure() extends BaseProcedure with ProcedureBuilde private def getHoodieCommitMetadata(timeline: HoodieTimeline, hoodieInstant: Option[HoodieInstant]): Option[HoodieCommitMetadata] = { if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { - Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) + Option(layout.getCommitMetadataSerDe().deserialize(hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.get), classOf[HoodieReplaceCommitMetadata])) } else { val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) - Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.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 b17b91a6d86d5..dbe6544f475ed 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 @@ -20,7 +20,7 @@ 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.HoodieTableMetaClient -import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline, TimelineLayout} +import org.apache.hudi.common.table.timeline.{CommitMetadataSerDe, HoodieInstant, HoodieTimeline, TimelineLayout} import org.apache.hudi.common.util.ClusteringUtils import org.apache.hudi.exception.HoodieException import org.apache.spark.sql.Row @@ -61,7 +61,7 @@ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilde val activeTimeline = metaClient.getActiveTimeline val timeline = activeTimeline.getCommitsTimeline.filterCompletedInstants val hoodieInstantOption = getCommitForInstant(metaClient, timeline, instantTime) - val commitMetadataOptional = getHoodieCommitMetadata(timeline, hoodieInstantOption) + val commitMetadataOptional = getHoodieCommitMetadata(metaClient.getCommitMetadataSerDe, timeline, hoodieInstantOption) if (commitMetadataOptional.isEmpty) { throw new HoodieException(s"Commit $instantTime not found in Commits $timeline.") @@ -93,14 +93,15 @@ class ShowCommitWriteStatsProcedure() extends BaseProcedure with ProcedureBuilde hoodieInstant } - private def getHoodieCommitMetadata(timeline: HoodieTimeline, hoodieInstant: Option[HoodieInstant]): Option[HoodieCommitMetadata] = { + private def getHoodieCommitMetadata(serDe: CommitMetadataSerDe, timeline: HoodieTimeline, hoodieInstant: Option[HoodieInstant]): Option[HoodieCommitMetadata] = { if (hoodieInstant.isDefined) { if (ClusteringUtils.isClusteringOrReplaceCommitAction(hoodieInstant.get.getAction)) { - Option(HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(hoodieInstant.get).get, + Option(serDe.deserialize(hoodieInstant.get, + timeline.getInstantContentStream(hoodieInstant.get), classOf[HoodieReplaceCommitMetadata])) } else { val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) - Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantDetails(hoodieInstant.get).get, + Option(layout.getCommitMetadataSerDe.deserialize(hoodieInstant.get, timeline.getInstantContentStream(hoodieInstant.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 b9a130d8eb1bd..74bfa3b6e90cf 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 @@ -101,7 +101,7 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) for (partitionWriteStat <- commitMetadata.getPartitionToWriteStats.entrySet.asScala) { for (hoodieWriteStat <- partitionWriteStat.getValue.asScala) { rows.add(Row( @@ -135,7 +135,7 @@ class ShowCommitsProcedure(includeExtraMetadata: Boolean) extends BaseProcedure val layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion) for (i <- 0 until newCommits.size) { val commit = newCommits.get(i) - val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantDetails(commit).get, classOf[HoodieCommitMetadata]) + val commitMetadata = layout.getCommitMetadataSerDe.deserialize(commit, timeline.getInstantContentStream(commit), classOf[HoodieCommitMetadata]) rows.add(Row(commit.requestedTime, commit.getCompletionTime, commit.getAction, commitMetadata.fetchTotalBytesWritten, commitMetadata.fetchTotalFilesInsert, commitMetadata.fetchTotalFilesUpdated, commitMetadata.fetchTotalPartitionsWritten, commitMetadata.fetchTotalRecordsWritten, commitMetadata.fetchTotalUpdateRecordsWritten, 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 bc9ded08c1ae7..5c772d6ed03ed 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 @@ -123,7 +123,7 @@ class ShowFileStatusProcedure extends BaseProcedure restoreInstant.find { instant => val hoodieRestoreMetadata = TimelineMetadataUtils.deserializeHoodieRestoreMetadata( - metaClient.getActiveTimeline.getInstantDetails(instant).get + metaClient.getActiveTimeline.getInstantContentStream(instant) ) val restoreMetadata = hoodieRestoreMetadata.getHoodieRestoreMetadata.values().asScala @@ -166,7 +166,7 @@ class ShowFileStatusProcedure extends BaseProcedure rollbackInstant.find { instant => val rollbackMetadata = - TimelineMetadataUtils.deserializeHoodieRollbackMetadata(timeline.getInstantDetails(instant).get()) + TimelineMetadataUtils.deserializeHoodieRollbackMetadata(timeline.getInstantContentStream(instant)) val partitionRollbackMetadata = rollbackMetadata.getPartitionMetadata partition.flatMap( p => Option.apply(partitionRollbackMetadata.get(p)).flatMap( @@ -190,7 +190,7 @@ class ShowFileStatusProcedure extends BaseProcedure reloadTimelineIfNecessary(timeline) cleanedInstant.find { instant => val cleanMetadata = TimelineMetadataUtils.deserializeHoodieCleanMetadata( - timeline.getInstantDetails(instant).get()) + timeline.getInstantContentStream(instant)) 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))) 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 ad9e4d08e6958..8f0198ee3340a 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 @@ -88,8 +88,8 @@ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with Pr limit: Int): Seq[Row] = { val rows = new util.ArrayList[Row] val instantGenerator = metaClient.getTimelineLayout.getInstantGenerator - val metadata = TimelineMetadataUtils.deserializeAvroMetadata(activeTimeline.getInstantDetails( - instantGenerator.createNewInstant(State.COMPLETED, ROLLBACK_ACTION, instantTime)).get, classOf[HoodieRollbackMetadata]) + val metadata = TimelineMetadataUtils.deserializeAvroMetadata(activeTimeline.getInstantContentStream( + instantGenerator.createNewInstant(State.COMPLETED, ROLLBACK_ACTION, instantTime)), classOf[HoodieRollbackMetadata]) metadata.getPartitionMetadata.asScala.toMap.iterator.foreach(entry => Stream .concat(entry._2.getSuccessDeleteFiles.asScala.map(f => (f, true)), @@ -108,7 +108,7 @@ class ShowRollbacksProcedure(showDetails: Boolean) extends BaseProcedure with Pr rollback.getInstants.iterator().asScala.foreach(instant => { try { - val metadata = TimelineMetadataUtils.deserializeAvroMetadata(activeTimeline.getInstantDetails(instant).get, + val metadata = TimelineMetadataUtils.deserializeAvroMetadata(activeTimeline.getInstantContentStream(instant), classOf[HoodieRollbackMetadata]) metadata.getCommitsRollback.iterator().asScala.foreach(c => { 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 3a4d6184b73e9..b347f0d7517d5 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 @@ -56,7 +56,7 @@ class StatsWriteAmplificationProcedure extends BaseProcedure with ProcedureBuild timeline.getInstants.iterator.asScala.foreach( instantTime => { var waf = "0" - val commit = layout.getCommitMetadataSerDe.deserialize(instantTime, activeTimeline.getInstantDetails(instantTime).get(), classOf[HoodieCommitMetadata]) + val commit = layout.getCommitMetadataSerDe.deserialize(instantTime, activeTimeline.getInstantContentStream(instantTime), classOf[HoodieCommitMetadata]) if (commit.fetchTotalUpdateRecordsWritten() > 0) { waf = df.format(commit.fetchTotalRecordsWritten().toFloat / commit.fetchTotalUpdateRecordsWritten()) } 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 80f993d37fa39..01d0480f3a9ea 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 @@ -194,7 +194,7 @@ class ValidateHoodieSyncProcedure extends BaseProcedure with ProcedureBuilder wi for (commit <- commitsToCatchup) { 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, + val c: HoodieCommitMetadata = target.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, timeline.getInstantContentStream(instant), classOf[HoodieCommitMetadata]) totalNew += c.fetchTotalRecordsWritten - c.fetchTotalUpdateRecordsWritten } diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java index 73e104db8d40b..520b98c204937 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestHoodieBackedMetadata.java @@ -1942,7 +1942,7 @@ public void testColStatsPrefixLookup() throws IOException { metaClient.getActiveTimeline().getInstants().forEach(entry -> { try { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(entry, metaClient.getActiveTimeline().getInstantDetails(entry).get(), HoodieCommitMetadata.class); + .deserialize(entry, metaClient.getActiveTimeline().getInstantContentStream(entry), HoodieCommitMetadata.class); String commitTime = entry.requestedTime(); if (!commitToPartitionsToFiles.containsKey(commitTime)) { commitToPartitionsToFiles.put(commitTime, new HashMap<>()); diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java index c9bca4cd1630c..08c76f7a74666 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/client/functional/TestRemoteFileSystemViewWithMetadataTable.java @@ -185,7 +185,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 = COMMIT_METADATA_SER_DE.deserialize(compactionCommit, - timeline.getInstantDetails(compactionCommit).get(), HoodieCommitMetadata.class); + timeline.getInstantContentStream(compactionCommit), HoodieCommitMetadata.class); List> partitionFileIdPairList = commitMetadata.getPartitionToWriteStats().entrySet().stream().flatMap( entry -> { 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 ff83e65b7a545..43124444a1368 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 @@ -75,7 +75,7 @@ public void testInertsWithEmptyCommitsHavingWriterSchemaAsNull() throws Exceptio HoodieTableMetaClient metaClient = createMetaClient(jsc, basePath); HoodieActiveTimeline timeline = metaClient.getActiveTimeline(); HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(timeline.lastInstant().get(), - timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + timeline.getInstantContentStream(timeline.lastInstant().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/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java index 006a1bd80c90b..3715921ac6d11 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/TestHoodieMergeOnReadTable.java @@ -578,7 +578,7 @@ public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Excep table = HoodieSparkTable.create(cfg, context()); HoodieInstant instantOne = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instantOne, - table.getActiveTimeline().getInstantDetails(instantOne).get(), HoodieCommitMetadata.class); + table.getActiveTimeline().getInstantContentStream(instantOne), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { for (HoodieWriteStat stat : pstat.getValue()) { @@ -611,7 +611,7 @@ public void testMetadataStatsOnCommit(Boolean rollbackUsingMarkers) throws Excep table = HoodieSparkTable.create(cfg, context()); HoodieInstant instant3 = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); metadata = metaClient.getCommitMetadataSerDe().deserialize(instant3, - table.getActiveTimeline().getInstantDetails(instant3).get(), HoodieCommitMetadata.class); + table.getActiveTimeline().getInstantContentStream(instant3), HoodieCommitMetadata.class); inserts = 0; upserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -651,7 +651,7 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { HoodieTable table = HoodieSparkTable.create(cfg, context()); HoodieInstant instantOne = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); HoodieCommitMetadata metadata = metaClient.getCommitMetadataSerDe().deserialize(instantOne, - table.getActiveTimeline().getInstantDetails(instantOne).get(), + table.getActiveTimeline().getInstantContentStream(instantOne), HoodieCommitMetadata.class); int inserts = 0; for (Map.Entry> pstat : metadata.getPartitionToWriteStats().entrySet()) { @@ -676,7 +676,7 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { table = HoodieSparkTable.create(cfg, context()); HoodieInstant instantTwo = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); metadata = metaClient.getCommitMetadataSerDe().deserialize(instantTwo, - table.getActiveTimeline().getInstantDetails(instantTwo).get(), + table.getActiveTimeline().getInstantContentStream(instantTwo), HoodieCommitMetadata.class); inserts = 0; int upserts = 0; @@ -703,7 +703,7 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { table = HoodieSparkTable.create(cfg, context()); HoodieInstant instantThree = table.getActiveTimeline().getCommitsTimeline().lastInstant().get(); HoodieCommitMetadata metadata1 = metaClient.getCommitMetadataSerDe().deserialize(instantThree, - table.getActiveTimeline().getInstantDetails(instantThree).get(), + table.getActiveTimeline().getInstantContentStream(instantThree), HoodieCommitMetadata.class); // Ensure that the metadata stats from the extra metadata of delta commits is copied over to the compaction commit @@ -727,7 +727,7 @@ public void testRollingStatsWithSmallFileHandling() throws Exception { table = HoodieSparkTable.create(cfg, context()); HoodieInstant instant = table.getActiveTimeline().getDeltaCommitTimeline().lastInstant().get(); metadata = metaClient.getCommitMetadataSerDe().deserialize(instant, - table.getActiveTimeline().getInstantDetails(instant).get(), + table.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); inserts = 0; upserts = 0; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java index e38062fa7ad7d..74c075ffb6314 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/table/action/rollback/TestMergeOnReadRollbackActionExecutor.java @@ -374,8 +374,7 @@ public void testRollbackForCanIndexLogFile() throws IOException { HoodieInstant instant = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.DELTA_COMMIT_ACTION, "001"); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant, table.getMetaClient().getCommitTimeline() - .getInstantDetails(instant) - .get(), + .getInstantContentStream(instant), HoodieCommitMetadata.class); List firstPartitionWriteStat = commitMetadata.getPartitionToWriteStats().get(DEFAULT_FIRST_PARTITION_PATH); assertEquals(2, firstPartitionWriteStat.size()); @@ -402,8 +401,7 @@ public void testRollbackForCanIndexLogFile() throws IOException { HoodieInstant instant1 = INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, newCommitTime); commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(instant1, table.getMetaClient().getCommitTimeline() - .getInstantDetails(instant1) - .get(), + .getInstantContentStream(instant1), HoodieCommitMetadata.class); assertTrue(commitMetadata.getPartitionToWriteStats().containsKey(DEFAULT_FIRST_PARTITION_PATH)); assertTrue(commitMetadata.getPartitionToWriteStats().containsKey(DEFAULT_SECOND_PARTITION_PATH)); diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterPartitionTTL.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterPartitionTTL.scala index 5d891cfa37d58..5339d87096bde 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterPartitionTTL.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriterPartitionTTL.scala @@ -86,7 +86,7 @@ class TestHoodieSparkSqlWriterPartitionTTL extends HoodieSparkWriterTestBase { val timeline = HoodieTestUtils.createMetaClient(tempBasePath).getActiveTimeline assert(timeline.getCompletedReplaceTimeline.getInstants.size() > 0) val replaceInstant = timeline.getCompletedReplaceTimeline.getInstants.get(0) - val replaceMetadata = TimelineMetadataUtils.deserializeReplaceCommitMetadata(timeline.getInstantDetails(replaceInstant).get()) + val replaceMetadata = TimelineMetadataUtils.deserializeReplaceCommitMetadata(timeline.getInstantContentStream(replaceInstant)) assert(replaceMetadata.getPartitionToReplaceFileIds.containsKey("part1")) } 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 754535ae0bcac..751f52b292619 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 @@ -246,9 +246,9 @@ class TestHoodieActiveTimeline extends HoodieSparkClientTestBase { .save(basePath) val metaClient: HoodieTableMetaClient = createMetaClient(basePath) val activeTimeline = metaClient.getActiveTimeline - assertNotNull(activeTimeline.getInstantDetails(activeTimeline.lastInstant().get())) + assertNotNull(activeTimeline.getInstantContentStream(activeTimeline.lastInstant().get())) try { - activeTimeline.getInstantDetails(HoodieTestUtils.INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, + activeTimeline.getInstantContentStream(HoodieTestUtils.INSTANT_GENERATOR.createNewInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.CLUSTERING_ACTION, metaClient.createNewInstantTime())) } catch { // org.apache.hudi.common.util.ClusteringUtils.getRequestedReplaceMetadata depends upon this behaviour diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala index 6d2ec1cbd8106..a7d5d2a12c7f9 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSourceStorage.scala @@ -334,7 +334,7 @@ class TestMORDataSourceStorage extends SparkClientFunctionalTestHarness { shouldBaseFileInstantTimeMatch: Boolean = true): List[HoodieLogFile] = { val instant = metaClient.getActiveTimeline.getDeltaCommitTimeline.lastInstant().get() val commitMetadata = metaClient.getCommitMetadataSerDe.deserialize( - instant, metaClient.getActiveTimeline.getInstantDetails(instant).get, + instant, metaClient.getActiveTimeline.getInstantContentStream(instant), classOf[HoodieCommitMetadata]) val logFileList: List[HoodieLogFile] = commitMetadata.getFileIdAndFullPaths(metaClient.getBasePath) .asScala.values 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 0e31499e05c56..85fc7ba424288 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 @@ -464,7 +464,7 @@ class TestPartitionStatsIndex extends PartitionStatsIndexTestBase { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantContentStream(instant), classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles(MetadataPartitionType.PARTITION_STATS.getPartitionPath) 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 4a140e8c4928a..858bf7507c01a 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 @@ -528,7 +528,7 @@ class TestRecordLevelIndex extends RecordLevelIndexTestBase { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantContentStream(instant), classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles(MetadataPartitionType.RECORD_INDEX.getPartitionPath) 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 d7fe8894cbd24..de84069ae12ae 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 @@ -681,7 +681,7 @@ class TestSecondaryIndexPruning extends SparkClientFunctionalTestHarness { val compactionTimeline = metadataTableFSView.getVisibleCommitsAndCompactionTimeline.filterCompletedAndCompactionInstants() val lastCompactionInstant = compactionTimeline .filter(JavaConversions.getPredicate((instant: HoodieInstant) => - metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantDetails(instant).get, classOf[HoodieCommitMetadata]) + metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, compactionTimeline.getInstantContentStream(instant), classOf[HoodieCommitMetadata]) .getOperationType == WriteOperationType.COMPACT)) .lastInstant() val compactionBaseFile = metadataTableFSView.getAllBaseFiles("secondary_index_idx_not_record_key_col") 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 3ce7982e88006..4ec2a58263394 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 @@ -95,7 +95,7 @@ abstract class HoodieCDCTestBase extends HoodieSparkClientTestBase { */ protected def hasCDCLogFile(instant: HoodieInstant): Boolean = { val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + metaClient.reloadActiveTimeline().getInstantContentStream(instant), classOf[HoodieCommitMetadata] ) val hoodieWriteStats = commitMetadata.getWriteStats.asScala @@ -111,7 +111,7 @@ abstract class HoodieCDCTestBase extends HoodieSparkClientTestBase { */ protected def getCDCLogFile(instant: HoodieInstant): List[String] = { val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(instant, - metaClient.reloadActiveTimeline().getInstantDetails(instant).get(), + metaClient.reloadActiveTimeline().getInstantContentStream(instant), classOf[HoodieCommitMetadata] ) commitMetadata.getWriteStats.asScala.flatMap(_.getCdcStats.asScala.keys).toList diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala index a4c361aa0eb69..7c5803e38d755 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/command/index/TestSecondaryIndex.scala @@ -446,7 +446,7 @@ class TestSecondaryIndex extends HoodieSparkSqlTestBase { // 3. A time travel query is performed with data skipping enabled. // 4. If it was to look up the secondary index, it would have skipped the data file, but the data file is still present. // 5. Time travel query should throw an exception in this case. - val firstCommitMetadata = deserializeCommitMetadata(metaClient.reloadActiveTimeline().getInstantDetails(firstInstant).get()) + val firstCommitMetadata = deserializeCommitMetadata(metaClient.reloadActiveTimeline().getInstantContentStream(firstInstant)) val partitionToWriteStats = firstCommitMetadata.getPartitionToWriteStats.asScala.mapValues(_.asScala.toList) // Find the path for the given fileId val matchingPath: Option[String] = partitionToWriteStats.values.flatten diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala index 0df0f89922bd9..85218236ae237 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/common/HoodieSparkSqlTestBase.scala @@ -383,7 +383,7 @@ object HoodieSparkSqlTestBase { val cleanInstant = metaClient.reloadActiveTimeline().getCleanerTimeline.filterCompletedInstants().lastInstant().get() TimelineMetadataUtils.deserializeHoodieCleanMetadata(metaClient - .getActiveTimeline.getInstantDetails(cleanInstant).get) + .getActiveTimeline.getInstantContentStream(cleanInstant)) } def validateTableConfig(storage: HoodieStorage, 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 7d28d37d600d0..f772e43777e3d 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 = metaClient.getCommitMetadataSerDe().deserialize(lastInstant, metaClient.getActiveTimeline.getInstantDetails(lastInstant).get(), classOf[HoodieCommitMetadata]) + val commitMetadata = metaClient.getCommitMetadataSerDe().deserialize(lastInstant, metaClient.getActiveTimeline.getInstantContentStream(lastInstant), classOf[HoodieCommitMetadata]) val schemaStr = commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY) val schema = new Schema.Parser().parse(schemaStr) val fields = schema.getFields.asScala.map(_.name()) @@ -500,8 +500,8 @@ class TestAlterTableDropPartition extends HoodieSparkSqlTestBase { // check schema val metaClient = createMetaClient(spark, s"${tmp.getCanonicalPath}/$tableName") val lastInstant = metaClient.getActiveTimeline.getCommitsTimeline.lastInstant() - val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(lastInstant.get(), metaClient.getActiveTimeline.getInstantDetails( - lastInstant.get()).get(), classOf[HoodieCommitMetadata]) + val commitMetadata = metaClient.getTimelineLayout.getCommitMetadataSerDe.deserialize(lastInstant.get(), + metaClient.getActiveTimeline.getInstantContentStream(lastInstant.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/procedure/TestClusteringProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestClusteringProcedure.scala index 2f3714cbb9a9c..56346ab90d5b6 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 @@ -833,7 +833,7 @@ class TestClusteringProcedure extends HoodieSparkProcedureTestBase { val layout = TimelineLayout.fromVersion(commitTimeline.getTimelineLayoutVersion) commitTimeline.getReverseOrderedInstants.toArray.foreach(instant => { val commitMetadata = layout.getCommitMetadataSerDe.deserialize(instant.asInstanceOf[HoodieInstant], - commitTimeline.getInstantDetails(instant.asInstanceOf[HoodieInstant]).get, classOf[HoodieCommitMetadata]) + commitTimeline.getInstantContentStream(instant.asInstanceOf[HoodieInstant]), 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/TestDropPartitionProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala index bc8c8ebeabbcf..87b4d2cbc0945 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestDropPartitionProcedure.scala @@ -60,8 +60,9 @@ class TestDropPartitionProcedure extends HoodieSparkProcedureTestBase { .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst() .get() - val partitions = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(), classOf[HoodieReplaceCommitMetadata]) + val partitions = metaClient.getCommitMetadataSerDe.deserialize(replaceCommitInstant, + metaClient.getActiveTimeline.getInstantContentStream(replaceCommitInstant), + classOf[HoodieReplaceCommitMetadata]) .getPartitionToReplaceFileIds .keySet() @@ -109,8 +110,9 @@ class TestDropPartitionProcedure extends HoodieSparkProcedureTestBase { .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst() .get() - val partitions = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(), classOf[HoodieReplaceCommitMetadata]) + val partitions = metaClient.getCommitMetadataSerDe.deserialize(replaceCommitInstant, + metaClient.getActiveTimeline.getInstantContentStream(replaceCommitInstant), + classOf[HoodieReplaceCommitMetadata]) .getPartitionToReplaceFileIds .keySet() @@ -157,8 +159,9 @@ class TestDropPartitionProcedure extends HoodieSparkProcedureTestBase { .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst() .get() - val partitions = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(), classOf[HoodieReplaceCommitMetadata]) + val partitions = metaClient.getCommitMetadataSerDe.deserialize(replaceCommitInstant, + metaClient.getActiveTimeline.getInstantContentStream(replaceCommitInstant), + classOf[HoodieReplaceCommitMetadata]) .getPartitionToReplaceFileIds .keySet() @@ -206,8 +209,9 @@ class TestDropPartitionProcedure extends HoodieSparkProcedureTestBase { .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst() .get() - val partitions = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(), classOf[HoodieReplaceCommitMetadata]) + val partitions = metaClient.getCommitMetadataSerDe + .deserialize(replaceCommitInstant, metaClient.getActiveTimeline.getInstantContentStream(replaceCommitInstant), + classOf[HoodieReplaceCommitMetadata]) .getPartitionToReplaceFileIds .keySet() diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala index 7c85011c5c8d0..3f1f0295bceb3 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/hudi/procedure/TestTruncateTableProcedure.scala @@ -94,8 +94,8 @@ class TestTruncateTableProcedure extends HoodieSparkProcedureTestBase { val replaceCommitInstant = metaClient.getActiveTimeline.getWriteTimeline .getCompletedReplaceTimeline.getReverseOrderedInstants.findFirst() .get() - val partitions = HoodieReplaceCommitMetadata - .fromBytes(metaClient.getActiveTimeline.getInstantDetails(replaceCommitInstant).get(), classOf[HoodieReplaceCommitMetadata]) + val partitions = metaClient.getCommitMetadataSerDe.deserialize(replaceCommitInstant, + metaClient.getActiveTimeline.getInstantContentStream(replaceCommitInstant), classOf[HoodieReplaceCommitMetadata]) .getPartitionToReplaceFileIds .keySet() //Step3: check number of truncated partitions and location startWith 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 c41e3c261c18e..54a7893ab8552 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 @@ -1579,7 +1579,7 @@ Pair hasCommittedLogFiles( HoodieInstant instant = completedInstantsTimeline.filter(i -> i.requestedTime().equals(instantTime)) .firstInstant().get(); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe().deserialize( - instant, completedInstantsTimeline.getInstantDetails(instant).get(), + instant, completedInstantsTimeline.getInstantContentStream(instant), HoodieCommitMetadata.class ); committedFilesMap.put( 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 9c867f804cad5..cac9f0862b5f2 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 @@ -57,7 +57,7 @@ public String getCheckpoint() throws HoodieException { .map(instant -> { try { HoodieCommitMetadata commitMetadata = anotherDsHoodieMetaClient.getCommitMetadataSerDe() - .deserialize(instant, anotherDsHoodieMetaClient.getActiveTimeline().getInstantDetails(instant).get(), + .deserialize(instant, anotherDsHoodieMetaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); return commitMetadata.getMetadata(CHECKPOINT_KEY); } catch (IOException e) { 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 d5f12d17e99ae..717c0656bb4d0 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 @@ -152,7 +152,7 @@ private static long getAvgSizeFromSampleWrites(JavaSparkContext jsc, String samp checkState(lastInstantOpt.isPresent(), "The only completed instant should be present in sample_writes table."); HoodieInstant instant = lastInstantOpt.get(); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instant, metaClient.getCommitTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, metaClient.getCommitTimeline().getInstantContentStream(instant), 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/StreamerCheckpointUtils.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamerCheckpointUtils.java index 289c769eb4067..8ab2bee6b3260 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamerCheckpointUtils.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/streamer/StreamerCheckpointUtils.java @@ -206,7 +206,7 @@ public static Option> getLatestInstantAndComm try { TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() - .deserialize(instant, timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, timeline.getInstantContentStream(instant), HoodieCommitMetadata.class); if (!StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieStreamer.CHECKPOINT_KEY)) || !StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieStreamer.CHECKPOINT_RESET_KEY)) || !StringUtils.isNullOrEmpty(commitMetadata.getMetadata(STREAMER_CHECKPOINT_KEY_V2)) 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 1c3260e442ed0..7e67b4afcfc0a 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 @@ -694,7 +694,7 @@ static HoodieInstant assertCommitMetadata(String expected, String tablePath, int HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieInstant lastInstant = timeline.lastInstant().get(); HoodieCommitMetadata commitMetadata = - meta.getCommitMetadataSerDe().deserialize(lastInstant, timeline.getInstantDetails(lastInstant).get(), HoodieCommitMetadata.class); + meta.getCommitMetadataSerDe().deserialize(lastInstant, timeline.getInstantContentStream(lastInstant), HoodieCommitMetadata.class); assertEquals(totalCommits, timeline.countInstants()); if (meta.getTableConfig().getTableVersion() == HoodieTableVersion.EIGHT) { assertEquals(expected, commitMetadata.getMetadata(StreamerCheckpointV2.STREAMER_CHECKPOINT_KEY_V2)); 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 e6331ce0239d9..6425453f0c031 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 @@ -945,9 +945,10 @@ public void testCleanerDeleteReplacedDataWithArchive(Boolean asyncClean) throws HoodieTimeline replacedTimeline = meta.reloadActiveTimeline().getCompletedReplaceTimeline(); Option firstReplaceHoodieInstant = replacedTimeline.nthFromLastInstant(1); assertTrue(firstReplaceHoodieInstant.isPresent()); - - Option firstReplaceHoodieInstantDetails = replacedTimeline.getInstantDetails(firstReplaceHoodieInstant.get()); - HoodieReplaceCommitMetadata firstReplaceMetadata = HoodieReplaceCommitMetadata.fromBytes(firstReplaceHoodieInstantDetails.get(), HoodieReplaceCommitMetadata.class); + HoodieReplaceCommitMetadata firstReplaceMetadata = meta.getCommitMetadataSerDe().deserialize( + firstReplaceHoodieInstant.get(), + replacedTimeline.getInstantContentStream(firstReplaceHoodieInstant.get()), + HoodieReplaceCommitMetadata.class); Map> partitionToReplaceFileIds = firstReplaceMetadata.getPartitionToReplaceFileIds(); String partitionName = null; List replacedFileIDs = null; @@ -1894,7 +1895,7 @@ public void testFilterDupes() throws Exception { // Ensure it is empty HoodieCommitMetadata commitMetadata = new DefaultCommitMetadataSerDe() - .deserialize(newLastFinished, mClient.getActiveTimeline().getInstantDetails(newLastFinished).get(), HoodieCommitMetadata.class); + .deserialize(newLastFinished, mClient.getActiveTimeline().getInstantContentStream(newLastFinished), HoodieCommitMetadata.class); System.out.println("New Commit Metadata=" + commitMetadata); assertTrue(commitMetadata.getPartitionToWriteStats().isEmpty()); @@ -2009,7 +2010,7 @@ private void assertValidSchemaAndOperationTypeInCommitMetadata(HoodieInstant ins WriteOperationType operationType) { try { HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instant, metaClient.getActiveTimeline().getInstantDetails(instant).get(), HoodieCommitMetadata.class); + .deserialize(instant, metaClient.getActiveTimeline().getInstantContentStream(instant), HoodieCommitMetadata.class); assertFalse(StringUtils.isNullOrEmpty(commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY))); assertEquals(operationType, commitMetadata.getOperationType()); } catch (IOException ioException) { @@ -2262,20 +2263,20 @@ public void testDeltaStreamerMultiwriterCheckpoint() throws Exception { ObjectMapper objectMapper = new ObjectMapper(); HoodieCommitMetadata commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instants.get(0), metaClient.getCommitsTimeline().getInstantDetails(instants.get(0)).get(), HoodieCommitMetadata.class); + .deserialize(instants.get(0), metaClient.getCommitsTimeline().getInstantContentStream(instants.get(0)), HoodieCommitMetadata.class); Map checkpointVals = objectMapper.readValue(commitMetadata.getExtraMetadata().get(HoodieDeltaStreamer.CHECKPOINT_KEY), Map.class); String parquetFirstcheckpoint = checkpointVals.get("parquet"); assertNotNull(parquetFirstcheckpoint); commitMetadata = metaClient.getCommitMetadataSerDe() - .deserialize(instants.get(1), metaClient.getCommitsTimeline().getInstantDetails(instants.get(1)).get(), HoodieCommitMetadata.class); + .deserialize(instants.get(1), metaClient.getCommitsTimeline().getInstantContentStream(instants.get(1)), 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 = metaClient.getCommitMetadataSerDe() - .deserialize(instants.get(2), metaClient.getCommitsTimeline().getInstantDetails(instants.get(2)).get(), HoodieCommitMetadata.class); + .deserialize(instants.get(2), metaClient.getCommitsTimeline().getInstantContentStream(instants.get(2)), 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 bb97de3d5f5cd..08d684237f165 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 @@ -149,7 +149,7 @@ void testUpsertsContinuousModeWithMultipleWritersForConflicts(HoodieTableType ta HoodieTimeline timeline = meta.reloadActiveTimeline().getCommitsTimeline().filterCompletedInstants(); TimelineLayout layout = TimelineLayout.fromVersion(timeline.getTimelineLayoutVersion()); HoodieCommitMetadata commitMetadata = layout.getCommitMetadataSerDe() - .deserialize(timeline.firstInstant().get(), timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + .deserialize(timeline.firstInstant().get(), timeline.getInstantContentStream(timeline.firstInstant().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())); @@ -217,7 +217,7 @@ void testUpsertsContinuousModeWithMultipleWritersWithoutConflicts(HoodieTableTyp HoodieTableMetaClient meta = createMetaClient(new HadoopStorageConfiguration(hadoopConf), tableBasePath); HoodieTimeline timeline = meta.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); HoodieCommitMetadata commitMetadata = meta.getCommitMetadataSerDe().deserialize( - timeline.firstInstant().get(), timeline.getInstantDetails(timeline.firstInstant().get()).get(), HoodieCommitMetadata.class); + timeline.firstInstant().get(), timeline.getInstantContentStream(timeline.firstInstant().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())); @@ -328,7 +328,7 @@ private static HoodieCommitMetadata getLatestMetadata(HoodieTableMetaClient meta HoodieTimeline timeline = meta.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants(); return meta.getCommitMetadataSerDe().deserialize( - timeline.firstInstant().get(), timeline.getInstantDetails(timeline.lastInstant().get()).get(), HoodieCommitMetadata.class); + timeline.firstInstant().get(), timeline.getInstantContentStream(timeline.lastInstant().get()), HoodieCommitMetadata.class); } private static TypedProperties prepareMultiWriterProps(HoodieStorage storage, String basePath,