From d924f181633a08aa9124aa211fa16fd19d1f03df Mon Sep 17 00:00:00 2001 From: Danny Chan Date: Tue, 29 Aug 2023 09:54:08 +0800 Subject: [PATCH] [HUDI-6539] New LSM tree style archived timeline (#9209) * replace the log based archived timeline with new parquet based timeline * the timeline is organized as an LSM tree, it has multiple versions for read/write snapshot isolation * maintain only one version pointer file, add file size limination to compaction strategy * write the manifest as JSON --- .../hudi/cli/commands/CompactionCommand.java | 19 +- .../apache/hudi/cli/commands/SparkMain.java | 2 +- .../commands/TestArchivedCommitsCommand.java | 2 +- .../hudi/cli/commands/TestCommitsCommand.java | 2 +- .../cli/commands/TestCompactionCommand.java | 2 +- .../client/BaseHoodieTableServiceClient.java | 1 + .../hudi/client/timeline/ActiveAction.java | 167 ++++++++ .../timeline/ActiveActionWithDetails.java | 112 ++++++ .../HoodieTimelineArchiver.java | 378 ++---------------- .../client/timeline/LSMTimelineWriter.java | 372 +++++++++++++++++ .../hudi/client/utils/ArchivalUtils.java | 2 +- .../utils/LegacyArchivedMetaEntryReader.java | 302 ++++++++++++++ .../client/utils/MetadataConversionUtils.java | 48 ++- .../hudi/config/HoodieArchivalConfig.java | 33 +- .../apache/hudi/config/HoodieWriteConfig.java | 12 +- .../org/apache/hudi/DummyActiveAction.java | 46 +++ .../TestConflictResolutionStrategyUtil.java | 2 +- .../TestLegacyArchivedMetaEntryReader.java | 146 +++++++ .../utils/TestMetadataConversionUtils.java | 164 +++++++- .../testutils/TestHoodieMetadataBase.java | 2 +- .../TestHoodieClientOnMergeOnReadStorage.java | 10 +- .../functional/TestHoodieMetadataBase.java | 2 +- .../hudi/io/TestHoodieTimelineArchiver.java | 270 ++++--------- .../org/apache/hudi/table/TestCleaner.java | 2 +- .../main/avro/HoodieLSMTimelineInstant.avsc | 54 +++ .../org/apache/hudi/common/fs/FSUtils.java | 29 ++ .../model/HoodieLSMTimelineManifest.java | 131 ++++++ .../common/table/HoodieTableMetaClient.java | 2 +- .../table/timeline/HoodieActiveTimeline.java | 2 +- .../timeline/HoodieArchivedTimeline.java | 346 ++++++---------- .../common/table/timeline/HoodieInstant.java | 4 +- .../common/table/timeline/LSMTimeline.java | 287 +++++++++++++ .../util/ArchivedInstantReadSchemas.java | 105 +++++ .../io/storage/HoodieAvroFileReaderBase.java | 7 +- .../io/storage/HoodieAvroHFileReader.java | 2 +- .../hudi/io/storage/HoodieAvroOrcReader.java | 2 +- .../io/storage/HoodieAvroParquetReader.java | 2 +- .../hudi/io/storage/HoodieFileWriter.java | 2 +- .../model/TestHoodieLSMTimelineManifest.java | 55 +++ .../table/timeline/TestLSMTimeline.java | 57 +++ .../common/testutils/FileCreateUtils.java | 17 +- .../common/testutils/HoodieTestTable.java | 30 +- .../compact/CompactionCommitTestSink.java | 2 +- .../apache/hudi/cli/ArchiveExecutorUtils.java | 2 +- .../TestSparkConsistentBucketClustering.java | 4 +- .../benchmark/LSMTimelineReadBenchmark.scala | 97 +++++ 46 files changed, 2453 insertions(+), 884 deletions(-) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveAction.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveActionWithDetails.java rename hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/{ => timeline}/HoodieTimelineArchiver.java (50%) create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java create mode 100644 hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java create mode 100644 hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java create mode 100644 hudi-common/src/main/avro/HoodieLSMTimelineInstant.avsc create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLSMTimelineManifest.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/table/timeline/LSMTimeline.java create mode 100644 hudi-common/src/main/java/org/apache/hudi/common/util/ArchivedInstantReadSchemas.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieLSMTimelineManifest.java create mode 100644 hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestLSMTimeline.java create mode 100644 hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala 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 c9cebb1b227f..0a21dd71d2b4 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 @@ -174,8 +174,8 @@ public String compactionShowArchived( HoodieTimeline.COMPACTION_ACTION, compactionInstantTime); try { archivedTimeline.loadCompactionDetailsInMemory(compactionInstantTime); - HoodieCompactionPlan compactionPlan = TimelineMetadataUtils.deserializeAvroRecordMetadata( - archivedTimeline.getInstantDetails(instant).get(), HoodieCompactionPlan.getClassSchema()); + HoodieCompactionPlan compactionPlan = + TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get()); return printCompaction(compactionPlan, sortByField, descending, limit, headerOnly, partition); } finally { archivedTimeline.clearInstantDetailsFromMemory(compactionInstantTime); @@ -365,17 +365,10 @@ Function compactionPlanReader( private HoodieCompactionPlan readCompactionPlanForArchivedTimeline(HoodieArchivedTimeline archivedTimeline, HoodieInstant instant) { - // filter inflight compaction - if (HoodieTimeline.COMPACTION_ACTION.equals(instant.getAction()) - && HoodieInstant.State.INFLIGHT.equals(instant.getState())) { - try { - return TimelineMetadataUtils.deserializeAvroRecordMetadata(archivedTimeline.getInstantDetails(instant).get(), - HoodieCompactionPlan.getClassSchema()); - } catch (Exception e) { - throw new HoodieException(e.getMessage(), e); - } - } else { - return null; + try { + return TimelineMetadataUtils.deserializeCompactionPlan(archivedTimeline.getInstantDetails(instant).get()); + } catch (Exception e) { + throw new HoodieException(e.getMessage(), e); } } diff --git a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java index 281ab3994f75..93c57940c43c 100644 --- a/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java +++ b/hudi-cli/src/main/java/org/apache/hudi/cli/commands/SparkMain.java @@ -20,9 +20,9 @@ import org.apache.hudi.DataSourceWriteOptions; import org.apache.hudi.cli.utils.SparkUtil; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.engine.HoodieEngineContext; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java index a34927ae0176..2df1dd3c2a3a 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestArchivedCommitsCommand.java @@ -25,7 +25,7 @@ import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; import org.apache.hudi.cli.testutils.HoodieTestCommitUtilities; import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java index 79f406be9b8c..d11f2617e390 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCommitsCommand.java @@ -26,7 +26,7 @@ import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; import org.apache.hudi.cli.testutils.HoodieTestReplaceCommitMetadataGenerator; import org.apache.hudi.cli.testutils.ShellEvaluationResultUtil; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieTableType; diff --git a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java index f1ea09470d35..a08aeea22e9a 100644 --- a/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java +++ b/hudi-cli/src/test/java/org/apache/hudi/cli/commands/TestCompactionCommand.java @@ -24,7 +24,7 @@ import org.apache.hudi.cli.TableHeader; import org.apache.hudi.cli.functional.CLIFunctionalTestHarness; import org.apache.hudi.cli.testutils.HoodieTestCommitMetadataGenerator; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.fs.NoOpConsistencyGuard; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java index 0af2ace25f09..ead1571ba268 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/BaseHoodieTableServiceClient.java @@ -28,6 +28,7 @@ import org.apache.hudi.avro.model.HoodieRollbackPlan; import org.apache.hudi.client.embedded.EmbeddedTimelineService; import org.apache.hudi.client.heartbeat.HeartbeatUtils; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.HoodiePendingRollbackInfo; import org.apache.hudi.common.data.HoodieData; import org.apache.hudi.common.engine.HoodieEngineContext; diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveAction.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveAction.java new file mode 100644 index 000000000000..ebe02cc723ff --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveAction.java @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.timeline; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; + +import javax.annotation.Nullable; + +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.Objects; + +/** + * A combination of instants covering action states: requested, inflight, completed. + */ +public class ActiveAction implements Serializable, Comparable { + private final HoodieInstant requested; + private final HoodieInstant inflight; + private final HoodieInstant completed; + + /** + * The constructor. + */ + protected ActiveAction(@Nullable HoodieInstant requested, @Nullable HoodieInstant inflight, HoodieInstant completed) { + this.requested = requested; + this.inflight = inflight; + this.completed = completed; + } + + public static ActiveAction fromInstants(List instants) { + ValidationUtils.checkArgument(instants.size() <= 3); + HoodieInstant requested = null; + HoodieInstant inflight = null; + HoodieInstant completed = null; + for (HoodieInstant instant : instants) { + if (instant.isRequested()) { + requested = instant; + } else if (instant.isInflight()) { + inflight = instant; + } else { + completed = instant; + } + } + return new ActiveAction(requested, inflight, Objects.requireNonNull(completed)); + } + + public List getPendingInstants() { + List instants = new ArrayList<>(2); + if (this.requested != null) { + instants.add(this.requested); + } + if (this.inflight != null) { + instants.add(this.inflight); + } + return instants; + } + + public HoodieInstant getCompleted() { + return completed; + } + + public String getAction() { + return this.completed.getAction(); + } + + /** + * A COMPACTION action eventually becomes COMMIT when completed. + */ + public String getPendingAction() { + return getPendingInstant().getAction(); + } + + public String getInstantTime() { + return this.completed.getTimestamp(); + } + + public String getCompletionTime() { + return this.completed.getStateTransitionTime(); + } + + public Option getCommitMetadata(HoodieTableMetaClient metaClient) { + Option content = metaClient.getActiveTimeline().getInstantDetails(this.completed); + if (content.isPresent() && content.get().length == 0) { + return Option.empty(); + } + return content; + } + + public Option getRequestedCommitMetadata(HoodieTableMetaClient metaClient) { + if (this.requested != null) { + Option requestedContent = metaClient.getActiveTimeline().getInstantDetails(this.requested); + if (!requestedContent.isPresent() || requestedContent.get().length == 0) { + return Option.empty(); + } else { + return requestedContent; + } + } else { + return Option.empty(); + } + } + + public Option getInflightCommitMetadata(HoodieTableMetaClient metaClient) { + if (this.inflight != null) { + Option inflightContent = metaClient.getActiveTimeline().getInstantDetails(this.inflight); + if (!inflightContent.isPresent() || inflightContent.get().length == 0) { + return Option.empty(); + } else { + return inflightContent; + } + } else { + return Option.empty(); + } + } + + public byte[] getCleanPlan(HoodieTableMetaClient metaClient) { + return metaClient.getActiveTimeline().readCleanerInfoAsBytes(getPendingInstant()).get(); + } + + public byte[] getCompactionPlan(HoodieTableMetaClient metaClient) { + return metaClient.getActiveTimeline().readCompactionPlanAsBytes(HoodieTimeline.getCompactionRequestedInstant(getInstantTime())).get(); + } + + public byte[] getLogCompactionPlan(HoodieTableMetaClient metaClient) { + return metaClient.getActiveTimeline().readCompactionPlanAsBytes(HoodieTimeline.getLogCompactionRequestedInstant(getInstantTime())).get(); + } + + protected HoodieInstant getPendingInstant() { + if (requested != null) { + return requested; + } else if (inflight != null) { + return inflight; + } else { + throw new AssertionError("Pending instant does not exist."); + } + } + + @Override + public int compareTo(ActiveAction other) { + return this.completed.getTimestamp().compareTo(other.completed.getTimestamp()); + } + + @Override + public String toString() { + return getCompleted().getTimestamp() + "__" + getCompleted().getAction(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveActionWithDetails.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveActionWithDetails.java new file mode 100644 index 000000000000..4a781eab4396 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/ActiveActionWithDetails.java @@ -0,0 +1,112 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.timeline; + +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.collection.Pair; + +import javax.annotation.Nullable; + +import java.util.List; +import java.util.Objects; + +/** + * A combination of instants covering action states: requested, inflight, completed. + * + *

It holds all the instant details besides the instants. + */ +public class ActiveActionWithDetails extends ActiveAction { + private final Option requestedDetails; + private final Option inflightDetails; + private final Option completedDetails; + + /** + * The constructor. + */ + protected ActiveActionWithDetails( + @Nullable HoodieInstant requested, + Option requestedDetails, + @Nullable HoodieInstant inflight, + Option inflightDetails, + HoodieInstant completed, + Option completedDetails) { + super(requested, inflight, completed); + this.requestedDetails = requestedDetails; + this.inflightDetails = inflightDetails; + this.completedDetails = completedDetails; + } + + public static ActiveActionWithDetails fromInstantAndDetails(List>> instantAndDetails) { + ValidationUtils.checkArgument(instantAndDetails.size() <= 3); + + HoodieInstant requested = null; + HoodieInstant inflight = null; + HoodieInstant completed = null; + + Option requestedDetails = Option.empty(); + Option inflightDetails = Option.empty(); + Option completedDetails = Option.empty(); + + for (Pair> instantAndDetail : instantAndDetails) { + HoodieInstant instant = instantAndDetail.getKey(); + Option details = instantAndDetail.getRight(); + if (instant.isRequested()) { + requested = instant; + requestedDetails = details; + } else if (instant.isInflight()) { + inflight = instant; + inflightDetails = details; + } else { + completed = instant; + completedDetails = details; + } + } + return new ActiveActionWithDetails(requested, requestedDetails, inflight, inflightDetails, Objects.requireNonNull(completed), completedDetails); + } + + public Option getCommitMetadata(HoodieTableMetaClient metaClient) { + return this.completedDetails; + } + + public Option getRequestedCommitMetadata(HoodieTableMetaClient metaClient) { + return this.requestedDetails; + } + + public Option getInflightCommitMetadata(HoodieTableMetaClient metaClient) { + return this.inflightDetails; + } + + public byte[] getCleanPlan(HoodieTableMetaClient metaClient) { + ValidationUtils.checkState(this.requestedDetails.isPresent(), "clean plan does not exist"); + return this.requestedDetails.get(); + } + + public byte[] getCompactionPlan(HoodieTableMetaClient metaClient) { + ValidationUtils.checkState(this.requestedDetails.isPresent(), "compaction plan does not exist"); + return this.requestedDetails.get(); + } + + public byte[] getLogCompactionPlan(HoodieTableMetaClient metaClient) { + ValidationUtils.checkState(this.requestedDetails.isPresent(), "log compaction plan does not exist"); + return this.requestedDetails.get(); + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java similarity index 50% rename from hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java rename to hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java index d4abfa82d59f..f6cc0c70b65e 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/HoodieTimelineArchiver.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/HoodieTimelineArchiver.java @@ -17,67 +17,41 @@ * under the License. */ -package org.apache.hudi.client; +package org.apache.hudi.client.timeline; -import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; -import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan; import org.apache.hudi.client.transaction.TransactionManager; -import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.engine.HoodieEngineContext; -import org.apache.hudi.common.fs.FSUtils; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.fs.StorageSchemes; -import org.apache.hudi.common.model.HoodieArchivedLogFile; -import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieAvroPayload; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodieRecord; -import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.log.HoodieLogFormat; -import org.apache.hudi.common.table.log.HoodieLogFormat.Writer; -import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; -import org.apache.hudi.common.table.log.block.HoodieLogBlock; -import org.apache.hudi.common.table.log.block.HoodieLogBlock.HeaderMetadataType; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; -import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; -import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.ClusteringUtils; import org.apache.hudi.common.util.CollectionUtils; import org.apache.hudi.common.util.CompactionUtils; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.config.HoodieWriteConfig; -import org.apache.hudi.exception.HoodieCommitException; import org.apache.hudi.exception.HoodieException; -import org.apache.hudi.exception.HoodieIOException; import org.apache.hudi.metadata.HoodieTableMetadata; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.compact.CompactionTriggerStrategy; import org.apache.hudi.table.marker.WriteMarkers; import org.apache.hudi.table.marker.WriteMarkersFactory; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.function.Consumer; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -94,63 +68,26 @@ public class HoodieTimelineArchiver { private static final Logger LOG = LoggerFactory.getLogger(HoodieTimelineArchiver.class); - private final Path archiveFilePath; private final HoodieWriteConfig config; - private Writer writer; private final int maxInstantsToKeep; private final int minInstantsToKeep; private final HoodieTable table; private final HoodieTableMetaClient metaClient; private final TransactionManager txnManager; + private final LSMTimelineWriter timelineWriter; + public HoodieTimelineArchiver(HoodieWriteConfig config, HoodieTable table) { this.config = config; this.table = table; this.metaClient = table.getMetaClient(); - this.archiveFilePath = HoodieArchivedTimeline.getArchiveLogPath(metaClient.getArchivePath()); this.txnManager = new TransactionManager(config, table.getMetaClient().getFs()); + this.timelineWriter = LSMTimelineWriter.getInstance(config, table); Pair minAndMaxInstants = getMinAndMaxInstantsToKeep(table, metaClient); this.minInstantsToKeep = minAndMaxInstants.getLeft(); this.maxInstantsToKeep = minAndMaxInstants.getRight(); } - private Writer openWriter() { - try { - if (this.writer == null) { - return HoodieLogFormat.newWriterBuilder().onParentPath(archiveFilePath.getParent()) - .withFileId(archiveFilePath.getName()).withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) - .withFs(metaClient.getFs()).overBaseCommit("").build(); - } else { - return this.writer; - } - } catch (IOException e) { - throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); - } - } - - public Writer reOpenWriter() { - try { - if (this.writer != null) { - this.writer.close(); - this.writer = null; - } - this.writer = openWriter(); - return writer; - } catch (IOException e) { - throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); - } - } - - private void close() { - try { - if (this.writer != null) { - this.writer.close(); - } - } catch (IOException e) { - throw new HoodieException("Unable to close HoodieLogFormat writer", e); - } - } - public boolean archiveIfRequired(HoodieEngineContext context) throws IOException { return archiveIfRequired(context, false); } @@ -164,230 +101,32 @@ public boolean archiveIfRequired(HoodieEngineContext context, boolean acquireLoc // there is no owner or instant time per se for archival. txnManager.beginTransaction(Option.empty(), Option.empty()); } - List instantsToArchive = getInstantsToArchive().collect(Collectors.toList()); - verifyLastMergeArchiveFilesIfNecessary(context); + // Sort again because the cleaning and rollback instants could break the sequence. + List instantsToArchive = getInstantsToArchive().sorted().collect(Collectors.toList()); boolean success = true; if (!instantsToArchive.isEmpty()) { - this.writer = openWriter(); LOG.info("Archiving instants " + instantsToArchive); - archive(context, instantsToArchive); + Consumer exceptionHandler = e -> { + if (this.config.isFailOnTimelineArchivingEnabled()) { + throw new HoodieException(e); + } + }; + this.timelineWriter.write(instantsToArchive, Option.of(action -> deleteAnyLeftOverMarkers(context, action)), Option.of(exceptionHandler)); LOG.info("Deleting archived instants " + instantsToArchive); success = deleteArchivedInstants(instantsToArchive, context); + // triggers compaction and cleaning only after archiving action + this.timelineWriter.compactAndClean(context); } else { LOG.info("No Instants to archive"); } - - if (shouldMergeSmallArchiveFiles()) { - mergeArchiveFilesIfNecessary(context); - } return success; } finally { - close(); if (acquireLock) { txnManager.endTransaction(Option.empty()); } } } - public boolean shouldMergeSmallArchiveFiles() { - return config.getArchiveMergeEnable() && !StorageSchemes.isAppendSupported(metaClient.getFs().getScheme()); - } - - /** - * Here Hoodie can merge the small archive files into a new larger one. - * Only used for filesystem which does not support append operation. - * The whole merge small archive files operation has four stages: - * 1. Build merge plan with merge candidates/merged file name infos. - * 2. Do merge. - * 3. Delete all the candidates. - * 4. Delete the merge plan. - * - * @param context HoodieEngineContext - * @throws IOException - */ - private void mergeArchiveFilesIfNecessary(HoodieEngineContext context) throws IOException { - Path planPath = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME); - // Flush remained content if existed and open a new write - reOpenWriter(); - // List all archive files - FileStatus[] fsStatuses = metaClient.getFs().globStatus( - new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - // Sort files by version suffix in reverse (implies reverse chronological order) - Arrays.sort(fsStatuses, new HoodieArchivedTimeline.ArchiveFileVersionComparator()); - - int archiveMergeFilesBatchSize = config.getArchiveMergeFilesBatchSize(); - long smallFileLimitBytes = config.getArchiveMergeSmallFileLimitBytes(); - - List mergeCandidate = getMergeCandidates(smallFileLimitBytes, fsStatuses); - - if (mergeCandidate.size() >= archiveMergeFilesBatchSize) { - List candidateFiles = mergeCandidate.stream().map(fs -> fs.getPath().toString()).collect(Collectors.toList()); - // before merge archive files build merge plan - String logFileName = computeLogFileName(); - buildArchiveMergePlan(candidateFiles, planPath, logFileName); - // merge archive files - mergeArchiveFiles(mergeCandidate); - // after merge, delete the small archive files. - deleteFilesParallelize(metaClient, candidateFiles, context, true); - LOG.info("Success to delete replaced small archive files."); - // finally, delete archiveMergePlan which means merging small archive files operation is successful. - metaClient.getFs().delete(planPath, false); - LOG.info("Success to merge small archive files."); - } - } - - /** - * Find the latest 'huge archive file' index as a break point and only check/merge newer archive files. - * Because we need to keep the original order of archive files which is important when loading archived instants with time filter. - * {@link HoodieArchivedTimeline} loadInstants(TimeRangeFilter filter, boolean loadInstantDetails, Function commitsFilter) - * - * @param smallFileLimitBytes small File Limit Bytes - * @param fsStatuses Sort by version suffix in reverse - * @return merge candidates - */ - private List getMergeCandidates(long smallFileLimitBytes, FileStatus[] fsStatuses) { - int index = 0; - for (; index < fsStatuses.length; index++) { - if (fsStatuses[index].getLen() > smallFileLimitBytes) { - break; - } - } - return Arrays.stream(fsStatuses).limit(index).collect(Collectors.toList()); - } - - /** - * Get final written archive file name based on storageSchemes which does not support append. - */ - private String computeLogFileName() throws IOException { - String logWriteToken = writer.getLogFile().getLogWriteToken(); - HoodieLogFile hoodieLogFile = writer.getLogFile().rollOver(metaClient.getFs(), logWriteToken); - return hoodieLogFile.getFileName(); - } - - /** - * Check/Solve if there is any failed and unfinished merge small archive files operation - * - * @param context HoodieEngineContext used for parallelize to delete small archive files if necessary. - * @throws IOException - */ - private void verifyLastMergeArchiveFilesIfNecessary(HoodieEngineContext context) throws IOException { - if (shouldMergeSmallArchiveFiles()) { - Path planPath = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME); - HoodieWrapperFileSystem fs = metaClient.getFs(); - // If plan exist, last merge small archive files was failed. - // we need to revert or complete last action. - if (fs.exists(planPath)) { - HoodieMergeArchiveFilePlan plan = null; - try { - plan = TimelineMetadataUtils.deserializeAvroMetadata(FileIOUtils.readDataFromPath(fs, planPath).get(), HoodieMergeArchiveFilePlan.class); - } catch (IOException e) { - LOG.warn("Parsing merge archive plan failed.", e); - // Reading partial plan file which means last merge action is failed during writing plan file. - fs.delete(planPath); - return; - } - Path mergedArchiveFile = new Path(metaClient.getArchivePath(), plan.getMergedArchiveFileName()); - List candidates = plan.getCandidate().stream().map(Path::new).collect(Collectors.toList()); - if (candidateAllExists(candidates)) { - // Last merge action is failed during writing merged archive file. - // But all the small archive files are not deleted. - // Revert last action by deleting mergedArchiveFile if existed. - if (fs.exists(mergedArchiveFile)) { - fs.delete(mergedArchiveFile, false); - } - } else { - // Last merge action is failed during deleting small archive files. - // But the merged files is completed. - // Try to complete last action - if (fs.exists(mergedArchiveFile)) { - deleteFilesParallelize(metaClient, plan.getCandidate(), context, true); - } - } - - fs.delete(planPath); - } - } - } - - /** - * If all the candidate small archive files existed, last merge operation was failed during writing the merged archive file. - * If at least one of candidate small archive files existed, the merged archive file was created and last operation was failed during deleting the small archive files. - */ - private boolean candidateAllExists(List candidates) throws IOException { - for (Path archiveFile : candidates) { - if (!metaClient.getFs().exists(archiveFile)) { - // candidate is deleted - return false; - } - } - return true; - } - - public void buildArchiveMergePlan(List compactCandidate, Path planPath, String compactedArchiveFileName) throws IOException { - LOG.info("Start to build archive merge plan."); - HoodieMergeArchiveFilePlan plan = HoodieMergeArchiveFilePlan.newBuilder() - .setCandidate(compactCandidate) - .setMergedArchiveFileName(compactedArchiveFileName) - .build(); - Option content = TimelineMetadataUtils.serializeAvroMetadata(plan, HoodieMergeArchiveFilePlan.class); - // building merge archive files plan. - FileIOUtils.createFileInPath(metaClient.getFs(), planPath, content); - LOG.info("Success to build archive merge plan"); - } - - public void mergeArchiveFiles(List compactCandidate) throws IOException { - LOG.info("Starting to merge small archive files."); - Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); - try { - List records = new ArrayList<>(); - for (FileStatus fs : compactCandidate) { - // Read the archived file - try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(), - new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) { - // Read the avro blocks - while (reader.hasNext()) { - HoodieAvroDataBlock blk = (HoodieAvroDataBlock) reader.next(); - blk.getRecordIterator(HoodieRecordType.AVRO).forEachRemaining(r -> records.add((IndexedRecord) r.getData())); - if (records.size() >= this.config.getCommitArchivalBatchSize()) { - writeToFile(wrapperSchema, records); - } - } - } - } - writeToFile(wrapperSchema, records); - } catch (Exception e) { - throw new HoodieCommitException("Failed to merge small archive files", e); - } finally { - writer.close(); - } - LOG.info("Success to merge small archive files."); - } - - private Map deleteFilesParallelize(HoodieTableMetaClient metaClient, List paths, HoodieEngineContext context, boolean ignoreFailed) { - - return FSUtils.parallelizeFilesProcess(context, - metaClient.getFs(), - config.getArchiveDeleteParallelism(), - pairOfSubPathAndConf -> { - Path file = new Path(pairOfSubPathAndConf.getKey()); - try { - FileSystem fs = metaClient.getFs(); - if (fs.exists(file)) { - return fs.delete(file, false); - } - return true; - } catch (IOException e) { - if (!ignoreFailed) { - throw new HoodieIOException("Failed to delete : " + file, e); - } else { - LOG.warn("Ignore failed deleting : " + file); - return true; - } - } - }, - paths); - } - private Stream getCleanInstantsToArchive() { HoodieTimeline cleanAndRollbackTimeline = table.getActiveTimeline() .getTimelineOfActions(CollectionUtils.createSet(HoodieTimeline.CLEAN_ACTION, HoodieTimeline.ROLLBACK_ACTION)).filterCompletedInstants(); @@ -397,7 +136,7 @@ private Stream getCleanInstantsToArchive() { if (hoodieInstants.size() > this.maxInstantsToKeep) { return hoodieInstants.subList(0, hoodieInstants.size() - this.minInstantsToKeep); } else { - return new ArrayList(); + return Collections.emptyList(); } }).flatMap(Collection::stream); } @@ -424,9 +163,7 @@ private Stream getCommitInstantsToArchive() throws IOException { LESSER_THAN, oldestPendingInstant.get().getTimestamp())).findFirst()); // Check if the completed instant is higher than the oldest inflight instant // in that case update the oldestCommitToRetain to oldestInflight commit time. - if (!completedCommitBeforeOldestPendingInstant.isPresent() - || HoodieTimeline.compareTimestamps(oldestPendingInstant.get().getTimestamp(), - LESSER_THAN, completedCommitBeforeOldestPendingInstant.get().getTimestamp())) { + if (!completedCommitBeforeOldestPendingInstant.isPresent()) { oldestCommitToRetain = oldestPendingInstant; } else { oldestCommitToRetain = completedCommitBeforeOldestPendingInstant; @@ -493,8 +230,7 @@ private Stream getCommitInstantsToArchive() throws IOException { } } - private Stream getInstantsToArchive() throws IOException { - Stream instants = Stream.concat(getCleanInstantsToArchive(), getCommitInstantsToArchive()); + private Stream getInstantsToArchive() throws IOException { if (config.isMetaserverEnabled()) { return Stream.empty(); } @@ -502,8 +238,10 @@ private Stream getInstantsToArchive() throws IOException { // For archiving and cleaning instants, we need to include intermediate state files if they exist HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); Map, List> groupByTsAction = rawActiveTimeline.getInstantsAsStream() - .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), - HoodieInstant.getComparableAction(i.getAction())))); + .collect(Collectors.groupingBy(i -> Pair.of(i.getTimestamp(), + HoodieInstant.getComparableAction(i.getAction())))); + + Stream instants = Stream.concat(getCleanInstantsToArchive(), getCommitInstantsToArchive()); // If metadata table is enabled, do not archive instants which are more recent than the last compaction on the // metadata table. @@ -552,30 +290,22 @@ private Stream getInstantsToArchive() throws IOException { } } - return instants.flatMap(hoodieInstant -> { + return instants.map(hoodieInstant -> { List instantsToStream = groupByTsAction.get(Pair.of(hoodieInstant.getTimestamp(), HoodieInstant.getComparableAction(hoodieInstant.getAction()))); - if (instantsToStream != null) { - return instantsToStream.stream(); - } else { - // if a concurrent writer archived the instant - return Stream.empty(); - } + return ActiveAction.fromInstants(instantsToStream); }); } - private boolean deleteArchivedInstants(List archivedInstants, HoodieEngineContext context) throws IOException { - LOG.info("Deleting instants " + archivedInstants); + private boolean deleteArchivedInstants(List activeActions, HoodieEngineContext context) { + LOG.info("Deleting instants " + activeActions); List pendingInstants = new ArrayList<>(); List completedInstants = new ArrayList<>(); - for (HoodieInstant instant : archivedInstants) { - if (instant.isCompleted()) { - completedInstants.add(instant); - } else { - pendingInstants.add(instant); - } + for (ActiveAction activeAction : activeActions) { + completedInstants.add(activeAction.getCompleted()); + pendingInstants.addAll(activeAction.getPendingInstants()); } context.setJobStatus(this.getClass().getSimpleName(), "Delete archived instants: " + config.getTableName()); @@ -604,52 +334,10 @@ private boolean deleteArchivedInstants(List archivedInstants, Hoo return true; } - public void archive(HoodieEngineContext context, List instants) throws HoodieCommitException { - try { - Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); - LOG.info("Wrapper schema " + wrapperSchema.toString()); - List records = new ArrayList<>(); - for (HoodieInstant hoodieInstant : instants) { - try { - deleteAnyLeftOverMarkers(context, hoodieInstant); - records.add(convertToAvroRecord(hoodieInstant)); - if (records.size() >= this.config.getCommitArchivalBatchSize()) { - writeToFile(wrapperSchema, records); - } - } catch (Exception e) { - LOG.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e); - if (this.config.isFailOnTimelineArchivingEnabled()) { - throw e; - } - } - } - writeToFile(wrapperSchema, records); - } catch (Exception e) { - throw new HoodieCommitException("Failed to archive commits", e); - } - } - - private void deleteAnyLeftOverMarkers(HoodieEngineContext context, HoodieInstant instant) { - WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, instant.getTimestamp()); + private void deleteAnyLeftOverMarkers(HoodieEngineContext context, ActiveAction activeAction) { + WriteMarkers writeMarkers = WriteMarkersFactory.get(config.getMarkersType(), table, activeAction.getInstantTime()); if (writeMarkers.deleteMarkerDir(context, config.getMarkersDeleteParallelism())) { - LOG.info("Cleaned up left over marker directory for instant :" + instant); + LOG.info("Cleaned up left over marker directory for instant :" + activeAction.getCompleted()); } } - - private void writeToFile(Schema wrapperSchema, List records) throws Exception { - if (records.size() > 0) { - Map header = new HashMap<>(); - header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); - final String keyField = table.getMetaClient().getTableConfig().getRecordKeyFieldProp(); - List indexRecords = records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); - HoodieAvroDataBlock block = new HoodieAvroDataBlock(indexRecords, header, keyField); - writer.appendBlock(block); - records.clear(); - } - } - - private IndexedRecord convertToAvroRecord(HoodieInstant hoodieInstant) - throws IOException { - return MetadataConversionUtils.createMetaWrapper(hoodieInstant, metaClient); - } } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java new file mode 100644 index 000000000000..f4866392d186 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/timeline/LSMTimelineWriter.java @@ -0,0 +1,372 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.client.timeline; + +import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; +import org.apache.hudi.client.utils.MetadataConversionUtils; +import org.apache.hudi.common.engine.HoodieEngineContext; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieFileFormat; +import org.apache.hudi.common.model.HoodieLSMTimelineManifest; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.LSMTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.ValidationUtils; +import org.apache.hudi.common.util.VisibleForTesting; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.io.storage.HoodieAvroParquetReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.hudi.io.storage.HoodieFileWriter; +import org.apache.hudi.io.storage.HoodieFileWriterFactory; +import org.apache.hudi.table.HoodieTable; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Comparator; +import java.util.List; +import java.util.Set; +import java.util.function.Consumer; +import java.util.stream.Collectors; + +/** + * A timeline writer which organizes the files as an LSM tree. + */ +public class LSMTimelineWriter { + private static final Logger LOG = LoggerFactory.getLogger(LSMTimelineWriter.class); + + public static final int FILE_LAYER_ZERO = 0; + + public static final long MAX_FILE_SIZE_IN_BYTES = 1024 * 1024 * 1000; + + private final HoodieWriteConfig config; + private final HoodieTable table; + private final HoodieTableMetaClient metaClient; + + private HoodieWriteConfig writeConfig; + + private LSMTimelineWriter(HoodieWriteConfig config, HoodieTable table) { + this.config = config; + this.table = table; + this.metaClient = table.getMetaClient(); + } + + public static LSMTimelineWriter getInstance(HoodieWriteConfig config, HoodieTable table) { + return new LSMTimelineWriter(config, table); + } + + /** + * Writes the list of active actions into the timeline. + * + * @param activeActions The active actions + * @param preWriteCallback The callback before writing each action + * @param exceptionHandler The handle for exception + */ + public void write( + List activeActions, + Option> preWriteCallback, + Option> exceptionHandler) throws HoodieCommitException { + ValidationUtils.checkArgument(!activeActions.isEmpty(), "The instant actions to write should not be empty"); + Path filePath = new Path(metaClient.getArchivePath(), + newFileName(activeActions.get(0).getInstantTime(), activeActions.get(activeActions.size() - 1).getInstantTime(), FILE_LAYER_ZERO)); + try (HoodieFileWriter writer = openWriter(filePath)) { + Schema wrapperSchema = HoodieLSMTimelineInstant.getClassSchema(); + LOG.info("Writing schema " + wrapperSchema.toString()); + for (ActiveAction activeAction : activeActions) { + try { + preWriteCallback.ifPresent(callback -> callback.accept(activeAction)); + // in local FS and HDFS, there could be empty completed instants due to crash. + final HoodieLSMTimelineInstant metaEntry = MetadataConversionUtils.createLSMTimelineInstant(activeAction, metaClient); + writer.write(metaEntry.getInstantTime(), new HoodieAvroIndexedRecord(metaEntry), wrapperSchema); + } catch (Exception e) { + LOG.error("Failed to write instant: " + activeAction.getInstantTime(), e); + exceptionHandler.ifPresent(handler -> handler.accept(e)); + } + } + updateManifest(filePath.getName()); + } catch (Exception e) { + throw new HoodieCommitException("Failed to write commits", e); + } + } + + /** + * Updates a manifest file. + * + *

3 steps: + *

    + *
  1. read the latest manifest version file;
  2. + *
  3. read the latest manifest file for valid files;
  4. + *
  5. add this new file to the existing file list from step2.
  6. + *
+ * + * @param fileToAdd New file name to add + */ + public void updateManifest(String fileToAdd) throws IOException { + updateManifest(Collections.emptyList(), fileToAdd); + } + + /** + * Updates a manifest file. + * + *

4 steps: + *

    + *
  1. read the latest manifest version file;
  2. + *
  3. read the latest manifest file for valid files;
  4. + *
  5. remove files to the existing file list from step2;
  6. + *
  7. add this new file to the existing file list from step2.
  8. + *
+ * + * @param filesToRemove File names to remove + * @param fileToAdd New file name to add + */ + public void updateManifest(List filesToRemove, String fileToAdd) throws IOException { + int latestVersion = LSMTimeline.latestSnapshotVersion(metaClient); + HoodieLSMTimelineManifest latestManifest = LSMTimeline.latestSnapshotManifest(metaClient, latestVersion); + HoodieLSMTimelineManifest newManifest = latestManifest.copy(filesToRemove); + newManifest.addFile(getFileEntry(fileToAdd)); + createManifestFile(newManifest, latestVersion); + } + + private void createManifestFile(HoodieLSMTimelineManifest manifest, int currentVersion) throws IOException { + byte[] content = manifest.toJsonString().getBytes(StandardCharsets.UTF_8); + // version starts from 1 and increases monotonically + int newVersion = currentVersion < 0 ? 1 : currentVersion + 1; + // create manifest file + final Path manifestFilePath = LSMTimeline.getManifestFilePath(metaClient, newVersion); + metaClient.getFs().createImmutableFileInPath(manifestFilePath, Option.of(content)); + // update version file + updateVersionFile(newVersion); + } + + private void updateVersionFile(int newVersion) throws IOException { + byte[] content = (String.valueOf(newVersion)).getBytes(StandardCharsets.UTF_8); + final Path versionFilePath = LSMTimeline.getVersionFilePath(metaClient); + metaClient.getFs().delete(versionFilePath, false); + metaClient.getFs().createImmutableFileInPath(versionFilePath, Option.of(content)); + } + + /** + * Compacts the small parquet files. + * + *

The parquet naming convention is: + * + *

${min_instant}_${max_instant}_${level}.parquet
+ * + *

The 'min_instant' and 'max_instant' represent the instant time range of the parquet file. + * The 'level' represents the number of the level where the file is located, currently we + * have no limit for the number of layers. + * + *

These parquet files composite as an LSM tree layout, one parquet file contains + * instant metadata entries with consecutive timestamp. Different parquet files may have + * overlapping with the instant time ranges. + * + *

+   *   t1_t2_0.parquet, t3_t4_0.parquet, ... t5_t6_0.parquet       L0 layer
+   *                          \            /
+   *                             \     /
+   *                                |
+   *                                V
+   *                          t3_t6_1.parquet                      L1 layer
+   * 
+ * + *

Compaction and cleaning: once the files number exceed a threshold(now constant 10) N, + * the oldest N files are then replaced with a compacted file in the next layer. + * A cleaning action is triggered right after the compaction. + * + * @param context HoodieEngineContext + */ + @VisibleForTesting + public void compactAndClean(HoodieEngineContext context) throws IOException { + // 1. List all the latest snapshot files + HoodieLSMTimelineManifest latestManifest = LSMTimeline.latestSnapshotManifest(metaClient); + int layer = 0; + // 2. triggers the compaction for L0 + Option compactedFileName = doCompact(latestManifest, layer); + while (compactedFileName.isPresent()) { + // 3. once a compaction had been executed for the current layer, + // continues to trigger compaction for the next layer. + latestManifest.addFile(getFileEntry(compactedFileName.get())); + compactedFileName = doCompact(latestManifest, ++layer); + } + + // cleaning + clean(context, layer); + } + + private Option doCompact(HoodieLSMTimelineManifest manifest, int layer) throws IOException { + // 1. list all the files that belong to current layer + List files = manifest.getFiles() + .stream().filter(file -> LSMTimeline.isFileFromLayer(file.getFileName(), layer)).collect(Collectors.toList()); + + int compactionBatchSize = config.getTimelineCompactionBatchSize(); + + if (files.size() >= compactionBatchSize) { + // 2. sort files by min instant time (implies ascending chronological order) + files.sort(HoodieLSMTimelineManifest.LSMFileEntry::compareTo); + List candidateFiles = getCandidateFiles(files, compactionBatchSize); + if (candidateFiles.size() < 2) { + // the file is too large to compact, returns early. + return Option.empty(); + } + String compactedFileName = compactedFileName(candidateFiles); + + // 3. compaction + compactFiles(candidateFiles, compactedFileName); + // 4. update the manifest file + updateManifest(candidateFiles, compactedFileName); + LOG.info("Finishes compaction of source files: " + candidateFiles); + return Option.of(compactedFileName); + } + return Option.empty(); + } + + public void compactFiles(List candidateFiles, String compactedFileName) { + LOG.info("Starting to compact source files."); + try (HoodieFileWriter writer = openWriter(new Path(metaClient.getArchivePath(), compactedFileName))) { + for (String fileName : candidateFiles) { + // Read the input source file + try (HoodieAvroParquetReader reader = (HoodieAvroParquetReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecord.HoodieRecordType.AVRO) + .getFileReader(metaClient.getHadoopConf(), new Path(metaClient.getArchivePath(), fileName))) { + // Read the meta entry + try (ClosableIterator iterator = reader.getIndexedRecordIterator(HoodieLSMTimelineInstant.getClassSchema(), HoodieLSMTimelineInstant.getClassSchema())) { + while (iterator.hasNext()) { + IndexedRecord record = iterator.next(); + writer.write(record.get(0).toString(), new HoodieAvroIndexedRecord(record), HoodieLSMTimelineInstant.getClassSchema()); + } + } + } + } + } catch (Exception e) { + throw new HoodieCommitException("Failed to compact source files", e); + } + } + + /** + * Checks whether there is any unfinished compaction operation. + * + * @param context HoodieEngineContext used for parallelize to delete obsolete files if necessary. + */ + public void clean(HoodieEngineContext context, int compactedVersions) throws IOException { + // if there are more than 3 version of snapshots, clean the oldest files. + List allSnapshotVersions = LSMTimeline.allSnapshotVersions(metaClient); + int numVersionsToKeep = 3 + compactedVersions; // should make the threshold configurable. + if (allSnapshotVersions.size() > numVersionsToKeep) { + allSnapshotVersions.sort((v1, v2) -> v2 - v1); + List versionsToKeep = allSnapshotVersions.subList(0, numVersionsToKeep); + Set filesToKeep = versionsToKeep.stream() + .flatMap(version -> LSMTimeline.latestSnapshotManifest(metaClient, version).getFileNames().stream()) + .collect(Collectors.toSet()); + // delete the manifest file first + List manifestFilesToClean = new ArrayList<>(); + Arrays.stream(LSMTimeline.listAllManifestFiles(metaClient)).forEach(fileStatus -> { + if (!versionsToKeep.contains(LSMTimeline.getManifestVersion(fileStatus.getPath().getName()))) { + manifestFilesToClean.add(fileStatus.getPath().toString()); + } + }); + FSUtils.deleteFilesParallelize(metaClient, manifestFilesToClean, context, config.getArchiveDeleteParallelism(), false); + // delete the data files + List dataFilesToClean = Arrays.stream(LSMTimeline.listAllMetaFiles(metaClient)) + .filter(fileStatus -> !filesToKeep.contains(fileStatus.getPath().getName())) + .map(fileStatus -> fileStatus.getPath().toString()) + .collect(Collectors.toList()); + FSUtils.deleteFilesParallelize(metaClient, dataFilesToClean, context, config.getArchiveDeleteParallelism(), false); + } + } + + private HoodieLSMTimelineManifest.LSMFileEntry getFileEntry(String fileName) throws IOException { + long fileLen = metaClient.getFs().getFileStatus(new Path(metaClient.getArchivePath(), fileName)).getLen(); + return HoodieLSMTimelineManifest.LSMFileEntry.getInstance(fileName, fileLen); + } + + /** + * Returns at most {@code filesBatch} number of source files + * restricted by the gross file size by 1GB. + */ + private List getCandidateFiles(List files, int filesBatch) throws IOException { + List candidates = new ArrayList<>(); + long totalFileLen = 0L; + for (int i = 0; i < filesBatch; i++) { + HoodieLSMTimelineManifest.LSMFileEntry fileEntry = files.get(i); + if (totalFileLen > MAX_FILE_SIZE_IN_BYTES) { + return candidates; + } + // we may also need to consider a single file that is very close to the threshold in size, + // to avoid the write amplification, + // for e.g, two 800MB files compact into a 1.6GB file. + totalFileLen += fileEntry.getFileLen(); + candidates.add(fileEntry.getFileName()); + } + return candidates; + } + + /** + * Returns a new file name. + */ + private static String newFileName(String minInstant, String maxInstant, int layer) { + return String.format("%s_%s_%d%s", minInstant, maxInstant, layer, HoodieFileFormat.PARQUET.getFileExtension()); + } + + /** + * Returns a new file name. + */ + @VisibleForTesting + public static String compactedFileName(List files) { + String minInstant = files.stream().map(LSMTimeline::getMinInstantTime) + .min(Comparator.naturalOrder()).get(); + String maxInstant = files.stream().map(LSMTimeline::getMaxInstantTime) + .max(Comparator.naturalOrder()).get(); + int currentLayer = LSMTimeline.getFileLayer(files.get(0)); + return newFileName(minInstant, maxInstant, currentLayer + 1); + } + + /** + * Get or create a writer config for parquet writer. + */ + private HoodieWriteConfig getOrCreateWriterConfig() { + if (this.writeConfig == null) { + this.writeConfig = HoodieWriteConfig.newBuilder() + .withProperties(this.config.getProps()) + .withPopulateMetaFields(false).build(); + } + return this.writeConfig; + } + + private HoodieFileWriter openWriter(Path filePath) { + try { + return HoodieFileWriterFactory.getFileWriter("", filePath, metaClient.getHadoopConf(), getOrCreateWriterConfig(), + HoodieLSMTimelineInstant.getClassSchema(), table.getTaskContextSupplier(), HoodieRecord.HoodieRecordType.AVRO); + } catch (IOException e) { + throw new HoodieException("Unable to initialize archiving writer", e); + } + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java index 1ef85f5ae358..9790f6342c59 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/ArchivalUtils.java @@ -20,7 +20,7 @@ package org.apache.hudi.client.utils; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; 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 new file mode 100644 index 000000000000..2e0134761f33 --- /dev/null +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/LegacyArchivedMetaEntryReader.java @@ -0,0 +1,302 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.avro.HoodieAvroUtils; +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.client.timeline.ActiveAction; +import org.apache.hudi.client.timeline.ActiveActionWithDetails; +import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.common.util.collection.Pair; +import org.apache.hudi.exception.HoodieIOException; + +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import javax.annotation.Nonnull; + +import java.io.IOException; +import java.io.Serializable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Comparator; +import java.util.Iterator; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +/** + * Tools used for migrating to new LSM tree style archived timeline. + */ +public class LegacyArchivedMetaEntryReader { + private static final Logger LOG = LoggerFactory.getLogger(LegacyArchivedMetaEntryReader.class); + + private static final Pattern ARCHIVE_FILE_PATTERN = + Pattern.compile("^\\.commits_\\.archive\\.([0-9]+).*"); + + public static final String MERGE_ARCHIVE_PLAN_NAME = "mergeArchivePlan"; + + private static final String ACTION_TYPE_KEY = "actionType"; + private static final String ACTION_STATE = "actionState"; + private static final String STATE_TRANSITION_TIME = "stateTransitionTime"; + + private final HoodieTableMetaClient metaClient; + + public LegacyArchivedMetaEntryReader(HoodieTableMetaClient metaClient) { + this.metaClient = metaClient; + } + + public ClosableIterator getActiveActionsIterator() { + return loadInstants(null); + } + + /** + * Reads the avro record for instant and details. + */ + private Pair> readInstant(GenericRecord record) { + final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString(); + final String action = record.get(ACTION_TYPE_KEY).toString(); + final String stateTransitionTime = (String) record.get(STATE_TRANSITION_TIME); + final Option details = getMetadataKey(action).map(key -> { + Object actionData = record.get(key); + if (actionData != null) { + if (action.equals(HoodieTimeline.COMPACTION_ACTION)) { + return HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData); + } else { + return actionData.toString().getBytes(StandardCharsets.UTF_8); + } + } + return null; + }); + HoodieInstant instant = new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, + instantTime, stateTransitionTime); + return Pair.of(instant,details); + } + + @Nonnull + private Option getMetadataKey(String action) { + switch (action) { + case HoodieTimeline.CLEAN_ACTION: + return Option.of("hoodieCleanMetadata"); + case HoodieTimeline.COMMIT_ACTION: + case HoodieTimeline.DELTA_COMMIT_ACTION: + return Option.of("hoodieCommitMetadata"); + case HoodieTimeline.ROLLBACK_ACTION: + return Option.of("hoodieRollbackMetadata"); + case HoodieTimeline.SAVEPOINT_ACTION: + return Option.of("hoodieSavePointMetadata"); + case HoodieTimeline.COMPACTION_ACTION: + case HoodieTimeline.LOG_COMPACTION_ACTION: + return Option.of("hoodieCompactionPlan"); + case HoodieTimeline.REPLACE_COMMIT_ACTION: + return Option.of("hoodieReplaceCommitMetadata"); + case HoodieTimeline.INDEXING_ACTION: + return Option.of("hoodieIndexCommitMetadata"); + default: + LOG.error(String.format("Unknown action in metadata (%s)", action)); + return Option.empty(); + } + } + + /** + * This is method to read selected instants. Do NOT use this directly use one of the helper methods above + * If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details + * If filter is specified, only the filtered instants are loaded + * If commitsFilter is specified, only the filtered records are loaded. + */ + private ClosableIterator loadInstants(HoodieArchivedTimeline.TimeRangeFilter filter) { + try { + // List all files + FileStatus[] fsStatuses = metaClient.getFs().globStatus( + new Path(metaClient.getArchivePath() + "/.commits_.archive*")); + + // Sort files by version suffix in reverse (implies reverse chronological order) + Arrays.sort(fsStatuses, new ArchiveLogVersionComparator()); + + ClosableIterator> itr = getRecordIterator(fsStatuses); + + return new ClosableIterator() { + private ActiveAction activeAction; + + private Pair> nextInstantAndDetail; + + @Override + public void close() { + itr.close(); + } + + @Override + public boolean hasNext() { + List>> instantAndDetails = new ArrayList<>(); + String lastInstantTime = null; + if (nextInstantAndDetail != null) { + instantAndDetails.add(nextInstantAndDetail); + lastInstantTime = nextInstantAndDetail.getKey().getTimestamp(); + nextInstantAndDetail = null; + } + while (itr.hasNext()) { + HoodieRecord record = itr.next(); + Pair> instantAndDetail = readInstant((GenericRecord) record.getData()); + String instantTime = instantAndDetail.getKey().getTimestamp(); + if (filter == null || filter.isInRange(instantTime)) { + if (lastInstantTime == null) { + instantAndDetails.add(instantAndDetail); + lastInstantTime = instantTime; + } else if (lastInstantTime.equals(instantTime)) { + instantAndDetails.add(instantAndDetail); + } else { + nextInstantAndDetail = instantAndDetail; + break; + } + } + } + if (!instantAndDetails.isEmpty()) { + this.activeAction = ActiveActionWithDetails.fromInstantAndDetails(instantAndDetails); + return true; + } + return false; + } + + @Override + public ActiveAction next() { + return this.activeAction; + } + }; + } catch (IOException e) { + throw new HoodieIOException( + "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); + } + } + + /** + * Returns the avro record iterator with given file statuses. + */ + private ClosableIterator> getRecordIterator(FileStatus[] fsStatuses) throws IOException { + return new ClosableIterator>() { + + final Iterator fsItr = Arrays.asList(fsStatuses).iterator(); + HoodieLogFormat.Reader reader; + ClosableIterator> recordItr; + + @Override + public void close() { + if (this.reader != null) { + closeLogFormatReader(reader); + } + } + + @Override + public boolean hasNext() { + if (recordItr != null && recordItr.hasNext()) { + return true; + } + // new reader if possible + if (reader != null) { + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieAvroDataBlock) { + HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block; + recordItr = avroBlock.getRecordIterator(HoodieRecord.HoodieRecordType.AVRO); + if (recordItr.hasNext()) { + return true; + } + } + } + // no records in the reader, close the reader + closeLogFormatReader(reader); + reader = null; + } + // new reader + while (fsItr.hasNext()) { + FileStatus fs = fsItr.next(); + try { + reader = HoodieLogFormat.newReader(metaClient.getFs(), + new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema()); + } catch (IOException ioe) { + throw new HoodieIOException("Error initializing the reader for archived log: " + fs.getPath(), ioe); + } + while (reader.hasNext()) { + HoodieLogBlock block = reader.next(); + if (block instanceof HoodieAvroDataBlock) { + HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block; + recordItr = avroBlock.getRecordIterator(HoodieRecord.HoodieRecordType.AVRO); + if (recordItr.hasNext()) { + return true; + } + } + } + } + return false; + } + + @Override + public HoodieRecord next() { + return this.recordItr.next(); + } + }; + } + + private void closeLogFormatReader(HoodieLogFormat.Reader reader) { + try { + reader.close(); + } catch (IOException ioe) { + throw new HoodieIOException("Error closing log format reader", ioe); + } + } + + /** + * Sort files by reverse order of version suffix in file name. + */ + public static class ArchiveLogVersionComparator implements Comparator, Serializable { + @Override + public int compare(FileStatus f1, FileStatus f2) { + return Integer.compare(getArchivedFileSuffix(f2), getArchivedFileSuffix(f1)); + } + } + + private static int getArchivedFileSuffix(FileStatus f) { + try { + Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(f.getPath().getName()); + if (fileMatcher.matches()) { + return Integer.parseInt(fileMatcher.group(1)); + } + } catch (NumberFormatException e) { + // log and ignore any format warnings + LOG.warn("error getting suffix for archived file: " + f.getPath()); + } + // return default value in case of any errors + return 0; + } +} diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java index cfd47ab2b374..ddae61f16e0f 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/client/utils/MetadataConversionUtils.java @@ -18,15 +18,14 @@ package org.apache.hudi.client.utils; -import com.fasterxml.jackson.databind.DeserializationFeature; -import com.fasterxml.jackson.databind.ObjectMapper; -import java.io.IOException; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; import org.apache.hudi.client.ReplaceArchivalHelper; +import org.apache.hudi.client.timeline.ActiveAction; import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; @@ -35,12 +34,19 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.LSMTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.TimelineUtils; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; + +import java.io.IOException; +import java.nio.ByteBuffer; + /** * Helper class to convert between different action related payloads and {@link HoodieArchivedMetaEntry}. */ @@ -137,6 +143,42 @@ public static HoodieArchivedMetaEntry createMetaWrapper(HoodieInstant hoodieInst return archivedMetaWrapper; } + public static HoodieLSMTimelineInstant createLSMTimelineInstant(ActiveAction activeAction, HoodieTableMetaClient metaClient) { + HoodieLSMTimelineInstant lsmTimelineInstant = new HoodieLSMTimelineInstant(); + lsmTimelineInstant.setInstantTime(activeAction.getInstantTime()); + lsmTimelineInstant.setCompletionTime(activeAction.getCompletionTime()); + lsmTimelineInstant.setAction(activeAction.getAction()); + activeAction.getCommitMetadata(metaClient).ifPresent(commitMetadata -> lsmTimelineInstant.setMetadata(ByteBuffer.wrap(commitMetadata))); + lsmTimelineInstant.setVersion(LSMTimeline.LSM_TIMELINE_INSTANT_VERSION_1); + switch (activeAction.getPendingAction()) { + case HoodieTimeline.CLEAN_ACTION: { + lsmTimelineInstant.setPlan(ByteBuffer.wrap(activeAction.getCleanPlan(metaClient))); + break; + } + case HoodieTimeline.REPLACE_COMMIT_ACTION: { + // 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 + activeAction.getRequestedCommitMetadata(metaClient).ifPresent(metadata -> lsmTimelineInstant.setPlan(ByteBuffer.wrap(metadata))); + // 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. + activeAction.getInflightCommitMetadata(metaClient).ifPresent(metadata -> lsmTimelineInstant.setPlan(ByteBuffer.wrap(metadata))); + break; + } + case HoodieTimeline.COMPACTION_ACTION: { + lsmTimelineInstant.setPlan(ByteBuffer.wrap(activeAction.getCompactionPlan(metaClient))); + break; + } + case HoodieTimeline.LOG_COMPACTION_ACTION: { + lsmTimelineInstant.setPlan(ByteBuffer.wrap(activeAction.getLogCompactionPlan(metaClient))); + break; + } + default: + // no operation + } + return lsmTimelineInstant; + } + public static HoodieArchivedMetaEntry createMetaWrapperForEmptyInstant(HoodieInstant hoodieInstant) { HoodieArchivedMetaEntry archivedMetaWrapper = new HoodieArchivedMetaEntry(); archivedMetaWrapper.setCommitTime(hoodieInstant.getTimestamp()); diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java index 23eaa276c553..0d6ae2180661 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieArchivalConfig.java @@ -88,24 +88,11 @@ public class HoodieArchivalConfig extends HoodieConfig { .withDocumentation("Archiving of instants is batched in best-effort manner, to pack more instants into a single" + " archive log. This config controls such archival batch size."); - public static final ConfigProperty ARCHIVE_MERGE_FILES_BATCH_SIZE = ConfigProperty - .key("hoodie.archive.merge.files.batch.size") + public static final ConfigProperty TIMELINE_COMPACTION_BATCH_SIZE = ConfigProperty + .key("hoodie.timeline.compaction.batch.size") .defaultValue(10) .markAdvanced() - .withDocumentation("The number of small archive files to be merged at once."); - - public static final ConfigProperty ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES = ConfigProperty - .key("hoodie.archive.merge.small.file.limit.bytes") - .defaultValue(20L * 1024 * 1024) - .markAdvanced() - .withDocumentation("This config sets the archive file size limit below which an archive file becomes a candidate to be selected as such a small file."); - - public static final ConfigProperty ARCHIVE_MERGE_ENABLE = ConfigProperty - .key("hoodie.archive.merge.enable") - .defaultValue(false) - .markAdvanced() - .withDocumentation("When enable, hoodie will auto merge several small archive files into larger one. It's" - + " useful when storage scheme doesn't support append operation."); + .withDocumentation("The number of small files to compact at once."); public static final ConfigProperty ARCHIVE_BEYOND_SAVEPOINT = ConfigProperty .key("hoodie.archive.beyond.savepoint") @@ -186,18 +173,8 @@ public HoodieArchivalConfig.Builder archiveCommitsWith(int minToKeep, int maxToK return this; } - public HoodieArchivalConfig.Builder withArchiveMergeFilesBatchSize(int number) { - archivalConfig.setValue(ARCHIVE_MERGE_FILES_BATCH_SIZE, String.valueOf(number)); - return this; - } - - public HoodieArchivalConfig.Builder withArchiveMergeSmallFileLimit(long size) { - archivalConfig.setValue(ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES, String.valueOf(size)); - return this; - } - - public HoodieArchivalConfig.Builder withArchiveMergeEnable(boolean enable) { - archivalConfig.setValue(ARCHIVE_MERGE_ENABLE, String.valueOf(enable)); + public HoodieArchivalConfig.Builder withTimelineCompactionBatchSize(int number) { + archivalConfig.setValue(TIMELINE_COMPACTION_BATCH_SIZE, String.valueOf(number)); return this; } diff --git a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java index 01b8fa559489..9840e9333621 100644 --- a/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java +++ b/hudi-client/hudi-client-common/src/main/java/org/apache/hudi/config/HoodieWriteConfig.java @@ -1481,8 +1481,8 @@ public int getMinCommitsToKeep() { return getInt(HoodieArchivalConfig.MIN_COMMITS_TO_KEEP); } - public int getArchiveMergeFilesBatchSize() { - return getInt(HoodieArchivalConfig.ARCHIVE_MERGE_FILES_BATCH_SIZE); + public int getTimelineCompactionBatchSize() { + return getInt(HoodieArchivalConfig.TIMELINE_COMPACTION_BATCH_SIZE); } public int getParquetSmallFileLimit() { @@ -1525,18 +1525,10 @@ public boolean isAutoClean() { return getBoolean(HoodieCleanConfig.AUTO_CLEAN); } - public boolean getArchiveMergeEnable() { - return getBooleanOrDefault(HoodieArchivalConfig.ARCHIVE_MERGE_ENABLE); - } - public boolean shouldArchiveBeyondSavepoint() { return getBooleanOrDefault(HoodieArchivalConfig.ARCHIVE_BEYOND_SAVEPOINT); } - public long getArchiveMergeSmallFileLimitBytes() { - return getLong(HoodieArchivalConfig.ARCHIVE_MERGE_SMALL_FILE_LIMIT_BYTES); - } - public boolean isAutoArchive() { return getBoolean(HoodieArchivalConfig.AUTO_ARCHIVE); } diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java new file mode 100644 index 000000000000..7b027369a797 --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/DummyActiveAction.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi; + +import org.apache.hudi.client.timeline.ActiveAction; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.util.Option; + +/** + * Instant triple for testing. + */ +public class DummyActiveAction extends ActiveAction { + private final byte[] commitMetadata; + + /** + * Only for testing purpose. + */ + public DummyActiveAction(HoodieInstant completed, byte[] commitMetadata) { + super(new HoodieInstant(HoodieInstant.State.REQUESTED, completed.getAction(), completed.getTimestamp()), + new HoodieInstant(HoodieInstant.State.INFLIGHT, completed.getAction(), completed.getTimestamp()), + completed); + this.commitMetadata = commitMetadata; + } + + @Override + public Option getCommitMetadata(HoodieTableMetaClient metaClient) { + return Option.of(this.commitMetadata); + } +} diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java index c11a29aa4f60..2569ff434eb8 100644 --- a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/transaction/TestConflictResolutionStrategyUtil.java @@ -222,7 +222,7 @@ public static void createPendingCompaction(String instantTime, HoodieTableMetaCl compactionPlan.setOperations(Arrays.asList(operation)); HoodieTestTable.of(metaClient) .addRequestedCompaction(instantTime, compactionPlan); - FileCreateUtils.createPendingInflightCompaction(metaClient.getBasePath(), instantTime); + FileCreateUtils.createInflightCompaction(metaClient.getBasePath(), instantTime); } public static void createCompleteCompaction(String instantTime, HoodieTableMetaClient metaClient) throws Exception { diff --git a/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java new file mode 100644 index 000000000000..3a8e320d7d9f --- /dev/null +++ b/hudi-client/hudi-client-common/src/test/java/org/apache/hudi/client/utils/TestLegacyArchivedMetaEntryReader.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.client.utils; + +import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; +import org.apache.hudi.client.timeline.ActiveAction; +import org.apache.hudi.common.model.HoodieArchivedLogFile; +import org.apache.hudi.common.model.HoodieAvroIndexedRecord; +import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.log.HoodieLogFormat; +import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; +import org.apache.hudi.common.table.log.block.HoodieLogBlock; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.testutils.HoodieTestTable; +import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.common.util.collection.ClosableIterator; +import org.apache.hudi.exception.HoodieCommitException; +import org.apache.hudi.exception.HoodieException; + +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.io.TempDir; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.MatcherAssert.assertThat; + +/** + * Test cases for {@link LegacyArchivedMetaEntryReader}. + */ +public class TestLegacyArchivedMetaEntryReader { + private static final Logger LOG = LoggerFactory.getLogger(TestLegacyArchivedMetaEntryReader.class); + + @TempDir + File tempFile; + + @Test + void testReadLegacyArchivedTimeline() throws Exception { + String tableName = "testTable"; + String tablePath = tempFile.getAbsolutePath() + Path.SEPARATOR + tableName; + HoodieTableMetaClient metaClient = HoodieTestUtils.init(new Configuration(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName); + prepareLegacyArchivedTimeline(metaClient); + LegacyArchivedMetaEntryReader reader = new LegacyArchivedMetaEntryReader(metaClient); + ClosableIterator iterator = reader.getActiveActionsIterator(); + List activeActions = new ArrayList<>(); + while (iterator.hasNext()) { + activeActions.add(iterator.next()); + } + assertThat(activeActions.stream().map(ActiveAction::getInstantTime).sorted().collect(Collectors.joining(",")), + is("00000001,00000002,00000003,00000004,00000005,00000006,00000007,00000008,00000009,00000010")); + } + + private void prepareLegacyArchivedTimeline(HoodieTableMetaClient metaClient) throws Exception { + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + for (int i = 1; i < 11; i++) { + String instantTime = String.format("%08d", i); + HoodieCommitMetadata metadata = testTable.createCommitMetadata(instantTime, WriteOperationType.INSERT, Arrays.asList("par1", "par2"), 10, false); + testTable.addCommit(instantTime, Option.of(metadata)); + } + List instants = new HoodieActiveTimeline(metaClient, false).getInstantsAsStream().sorted().collect(Collectors.toList()); + // archive 2 times to have 2 log files. + archive(metaClient, instants.subList(0, instants.size() / 2)); + archive(metaClient, instants.subList(instants.size() / 2, instants.size())); + } + + private HoodieLogFormat.Writer openWriter(HoodieTableMetaClient metaClient) { + try { + return HoodieLogFormat.newWriterBuilder().onParentPath(new Path(metaClient.getArchivePath())) + .withFileId("commits").withFileExtension(HoodieArchivedLogFile.ARCHIVE_EXTENSION) + .withFs(metaClient.getFs()).overBaseCommit("").build(); + } catch (IOException e) { + throw new HoodieException("Unable to initialize HoodieLogFormat writer", e); + } + } + + public void archive(HoodieTableMetaClient metaClient, List instants) throws HoodieCommitException { + try (HoodieLogFormat.Writer writer = openWriter(metaClient)) { + Schema wrapperSchema = HoodieArchivedMetaEntry.getClassSchema(); + LOG.info("Wrapper schema " + wrapperSchema.toString()); + List records = new ArrayList<>(); + for (HoodieInstant hoodieInstant : instants) { + try { + records.add(convertToAvroRecord(hoodieInstant, metaClient)); + } catch (Exception e) { + LOG.error("Failed to archive commits, .commit file: " + hoodieInstant.getFileName(), e); + throw e; + } + } + writeToFile(metaClient, wrapperSchema, records, writer); + } catch (Exception e) { + throw new HoodieCommitException("Failed to archive commits", e); + } + } + + private void writeToFile(HoodieTableMetaClient metaClient, Schema wrapperSchema, List records, HoodieLogFormat.Writer writer) throws Exception { + if (records.size() > 0) { + Map header = new HashMap<>(); + header.put(HoodieLogBlock.HeaderMetadataType.SCHEMA, wrapperSchema.toString()); + final String keyField = metaClient.getTableConfig().getRecordKeyFieldProp(); + List indexRecords = records.stream().map(HoodieAvroIndexedRecord::new).collect(Collectors.toList()); + HoodieAvroDataBlock block = new HoodieAvroDataBlock(indexRecords, header, keyField); + writer.appendBlock(block); + records.clear(); + } + } + + private IndexedRecord convertToAvroRecord(HoodieInstant hoodieInstant, HoodieTableMetaClient metaClient) + throws IOException { + return MetadataConversionUtils.createMetaWrapper(hoodieInstant, metaClient); + } +} 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 3938df3f3afd..a4f6531007a2 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 @@ -18,18 +18,6 @@ package org.apache.hudi.utils; -import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertNull; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Random; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -37,10 +25,12 @@ import org.apache.hudi.avro.model.HoodieClusteringGroup; import org.apache.hudi.avro.model.HoodieClusteringPlan; import org.apache.hudi.avro.model.HoodieInstantInfo; +import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; import org.apache.hudi.avro.model.HoodieSliceInfo; +import org.apache.hudi.client.timeline.ActiveAction; import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.model.HoodieCleaningPolicy; @@ -48,19 +38,39 @@ import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; import org.apache.hudi.common.testutils.HoodieTestTable; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.CleanerUtils; +import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; + import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.stream.Collectors; + +import static org.apache.hudi.common.util.CleanerUtils.convertCleanMetadata; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNull; + public class TestMetadataConversionUtils extends HoodieCommonTestHarness { @BeforeEach @@ -74,8 +84,8 @@ public void testCompletedClean() throws Exception { createCleanMetadata(newCommitTime); HoodieArchivedMetaEntry metaEntry = MetadataConversionUtils.createMetaWrapper( new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, newCommitTime), metaClient); - assertEquals(metaEntry.getActionState(), State.COMPLETED.toString()); - assertEquals(metaEntry.getHoodieCleanMetadata().getStartCleanTime(), newCommitTime); + assertEquals(State.COMPLETED.toString(), metaEntry.getActionState()); + assertEquals(newCommitTime, metaEntry.getHoodieCleanMetadata().getStartCleanTime()); } @Test @@ -167,6 +177,124 @@ public void testConvertCommitMetadata() { assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); } + // ------------------------------------------------------------------------- + // BEGIN: test cases for HoodieArchivedInstant conversion. + // ------------------------------------------------------------------------- + + @Test + public void testArchivedClean() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCleanMetadata(newCommitTime); + // test conversion to archived instant + HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); + assertEquals(newCommitTime, archived.getInstantTime()); + assertEquals(HoodieTimeline.CLEAN_ACTION, archived.getAction()); + assertDoesNotThrow(() -> CleanerUtils.getCleanerMetadata(metaClient, archived.getMetadata().array())); + assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeCleanerPlan(archived.getPlan().array())); + } + + @Test + public void testArchivedReplace() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE, true); + // test conversion to archived instant + 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())); + } + + @Test + public void testArchivedInsertOverwriteWithoutClustering() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE, false); + // test conversion to archived instant + 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(() -> HoodieCommitMetadata.fromBytes(archived.getPlan().array(), HoodieCommitMetadata.class), + "Insert overwrite without clustering should have a plan"); + + String newCommitTime2 = HoodieTestTable.makeNewCommitTime(); + createReplace(newCommitTime2, WriteOperationType.INSERT_OVERWRITE_TABLE, false); + // test conversion to archived instant + 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(() -> HoodieCommitMetadata.fromBytes(archived2.getPlan().array(), HoodieCommitMetadata.class), + "Insert overwrite table without clustering should have a plan"); + } + + @Test + public void testArchivedInsertOverwriteWithClustering() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createReplace(newCommitTime, WriteOperationType.INSERT_OVERWRITE, true); + // test conversion to archived instant + 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(); + createReplace(newCommitTime2, WriteOperationType.INSERT_OVERWRITE_TABLE, true); + // test conversion to archived instant + 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())); + } + + @Test + public void testArchivedCommit() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCommitMetadata(newCommitTime); + HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); + assertEquals(newCommitTime, archived.getInstantTime()); + assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); + assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieCommitMetadata.class)); + } + + @Test + public void testArchivedDeltaCommit() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createDeltaCommitMetadata(newCommitTime); + HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); + assertEquals(newCommitTime, archived.getInstantTime()); + assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, archived.getAction()); + assertNull(archived.getMetadata()); + } + + @Test + public void testArchivedRollback() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createRollbackMetadata(newCommitTime); + HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); + assertEquals(newCommitTime, archived.getInstantTime()); + assertEquals(HoodieTimeline.ROLLBACK_ACTION, archived.getAction()); + assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeHoodieRollbackMetadata(archived.getMetadata().array())); + assertNull(archived.getPlan()); + } + + @Test + public void testArchivedCompaction() throws Exception { + String newCommitTime = HoodieTestTable.makeNewCommitTime(); + createCompactionMetadata(newCommitTime); + HoodieLSMTimelineInstant archived = MetadataConversionUtils.createLSMTimelineInstant(getActiveInstant(newCommitTime), metaClient); + assertEquals(newCommitTime, archived.getInstantTime()); + assertEquals(HoodieTimeline.COMMIT_ACTION, archived.getAction()); + assertDoesNotThrow(() -> HoodieCommitMetadata.fromBytes(archived.getMetadata().array(), HoodieCommitMetadata.class)); + assertDoesNotThrow(() -> CompactionUtils.getCompactionPlan(metaClient, Option.of(archived.getPlan().array()))); + } + + // ------------------------------------------------------------------------- + // END: test cases for HoodieArchivedInstant conversion. + // ------------------------------------------------------------------------- + private void createCompactionMetadata(String instantTime) throws Exception { String fileId1 = "file-" + instantTime + "-1"; String fileId2 = "file-" + instantTime + "-2"; @@ -176,7 +304,7 @@ private void createCompactionMetadata(String instantTime) throws Exception { commitMetadata.setOperationType(WriteOperationType.COMPACT); commitMetadata.setCompacted(true); HoodieTestTable.of(metaClient) - .addCommit(instantTime, Option.of(commitMetadata)) + .addCompaction(instantTime, commitMetadata) .withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); } @@ -273,4 +401,10 @@ HoodieTestUtils.DEFAULT_PARTITION_PATHS[new Random().nextInt(HoodieTestUtils.DEF HoodieCleanMetadata cleanMetadata = convertCleanMetadata(instantTime, Option.of(0L), Collections.singletonList(cleanStats)); HoodieTestTable.of(metaClient).addClean(instantTime, cleanerPlan, cleanMetadata); } + + private ActiveAction getActiveInstant(String instantTime) { + HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); + List instants = rawActiveTimeline.getInstantsAsStream().filter(instant -> instant.getTimestamp().equals(instantTime)).collect(Collectors.toList()); + return ActiveAction.fromInstants(instants); + } } diff --git a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java index e7f13991addc..84e6c2cbabf3 100644 --- a/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-java-client/src/test/java/org/apache/hudi/testutils/TestHoodieMetadataBase.java @@ -18,8 +18,8 @@ package org.apache.hudi.testutils; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.common.HoodieJavaEngineContext; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java index be979c892f32..001e56d97bc1 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieClientOnMergeOnReadStorage.java @@ -524,14 +524,8 @@ public void testArchivalOnLogCompaction() throws Exception { if (instants == null) { continue; } - assertEquals(3, instants.size()); - for (HoodieInstant instant: instants) { - if (instant.isCompleted()) { - assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, instant.getAction()); - } else { - assertEquals(HoodieTimeline.LOG_COMPACTION_ACTION, instant.getAction()); - } - } + assertEquals(1, instants.size()); + assertEquals(HoodieTimeline.DELTA_COMMIT_ACTION, instants.get(0).getAction()); logCompactionInstantArchived = true; } assertTrue(logCompactionInstantArchived); diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java index e0a00c24e927..b9fbeab25826 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/client/functional/TestHoodieMetadataBase.java @@ -18,7 +18,7 @@ package org.apache.hudi.client.functional; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.model.HoodieCommitMetadata; diff --git a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java index f49f3d5920a8..df8348ccf901 100644 --- a/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java +++ b/hudi-client/hudi-spark-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiver.java @@ -20,7 +20,8 @@ import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieSavepointMetadata; -import org.apache.hudi.client.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; +import org.apache.hudi.client.timeline.LSMTimelineWriter; import org.apache.hudi.client.transaction.lock.InProcessLockProvider; import org.apache.hudi.client.utils.MetadataConversionUtils; import org.apache.hudi.common.config.HoodieMetadataConfig; @@ -28,17 +29,19 @@ import org.apache.hudi.common.model.HoodieCleaningPolicy; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy; +import org.apache.hudi.common.model.HoodieLSMTimelineManifest; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteConcurrencyMode; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.log.HoodieLogFormat; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.table.timeline.LSMTimeline; +import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; import org.apache.hudi.common.table.view.FileSystemViewStorageConfig; import org.apache.hudi.common.testutils.HoodieMetadataTestTable; import org.apache.hudi.common.testutils.HoodieTestDataGenerator; @@ -62,7 +65,6 @@ import org.apache.hudi.testutils.HoodieSparkClientTestHarness; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Disabled; @@ -105,9 +107,9 @@ import static org.apache.hudi.common.testutils.HoodieTestUtils.createCompactionCommitInMetadataTable; import static org.apache.hudi.config.HoodieArchivalConfig.ARCHIVE_BEYOND_SAVEPOINT; import static org.apache.hudi.metadata.HoodieTableMetadata.SOLO_COMMIT_TIMESTAMP; +import static org.junit.jupiter.api.Assertions.assertDoesNotThrow; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class TestHoodieTimelineArchiver extends HoodieSparkClientTestHarness { @@ -135,7 +137,7 @@ public void init(HoodieTableType tableType) throws Exception { hadoopConf.addResource(wrapperFs.getConf()); } - private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) throws IOException { + private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) { if (enableMetadataTable) { metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context); // reload because table configs could have been updated @@ -162,7 +164,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int maxDeltaCommitsMetadataTable, HoodieTableType tableType) throws Exception { return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, - maxDeltaCommits, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200, + maxDeltaCommits, maxDeltaCommitsMetadataTable, tableType, 10, HoodieFailedWritesCleaningPolicy.EAGER, WriteConcurrencyMode.SINGLE_WRITER); } @@ -172,7 +174,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int maxDeltaCommitsMetadataTable, HoodieTableType tableType) throws Exception { return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, - 5, maxDeltaCommitsMetadataTable, tableType, false, 10, 209715200, + 5, maxDeltaCommitsMetadataTable, tableType, 10, HoodieFailedWritesCleaningPolicy.EAGER, WriteConcurrencyMode.SINGLE_WRITER); } @@ -180,11 +182,9 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable, - boolean enableArchiveMerge, - int archiveFilesBatch, - long size) throws Exception { + int archiveFilesBatch) throws Exception { return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, 5, - maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE, enableArchiveMerge, archiveFilesBatch, size, + maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE, archiveFilesBatch, HoodieFailedWritesCleaningPolicy.EAGER, WriteConcurrencyMode.SINGLE_WRITER); } @@ -194,9 +194,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int maxDeltaCommits, int maxDeltaCommitsMetadataTable, HoodieTableType tableType, - boolean enableArchiveMerge, int archiveFilesBatch, - long size, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, WriteConcurrencyMode writeConcurrencyMode) throws Exception { return initTestTableAndGetWriteConfig( @@ -206,9 +204,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, maxDeltaCommits, maxDeltaCommitsMetadataTable, tableType, - enableArchiveMerge, archiveFilesBatch, - size, failedWritesCleaningPolicy, writeConcurrencyMode, ARCHIVE_BEYOND_SAVEPOINT.defaultValue()); @@ -220,9 +216,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int maxDeltaCommits, int maxDeltaCommitsMetadataTable, HoodieTableType tableType, - boolean enableArchiveMerge, int archiveFilesBatch, - long size, HoodieFailedWritesCleaningPolicy failedWritesCleaningPolicy, WriteConcurrencyMode writeConcurrencyMode, boolean archiveProceedBeyondSavepoints) throws Exception { @@ -231,9 +225,7 @@ private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2) .withCleanConfig(HoodieCleanConfig.newBuilder().retainCommits(1).withFailedWritesCleaningPolicy(failedWritesCleaningPolicy).build()) .withArchivalConfig(HoodieArchivalConfig.newBuilder() - .withArchiveMergeEnable(enableArchiveMerge) - .withArchiveMergeFilesBatchSize(archiveFilesBatch) - .withArchiveMergeSmallFileLimit(size) + .withTimelineCompactionBatchSize(archiveFilesBatch) .archiveCommitsWith(minArchivalCommits, maxArchivalCommits) .withArchiveBeyondSavepoint(archiveProceedBeyondSavepoints).build()) .withCompactionConfig(HoodieCompactionConfig.newBuilder() @@ -454,7 +446,7 @@ protected static HoodieCommitMetadata generateCommitMetadata( public void testSavepointWithArchival(boolean archiveBeyondSavepoint) throws Exception { boolean enableMetadata = false; HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 5, 2, HoodieTableType.COPY_ON_WRITE, - false, 10, 209715200, HoodieFailedWritesCleaningPolicy.EAGER, WriteConcurrencyMode.SINGLE_WRITER, archiveBeyondSavepoint); + 10, HoodieFailedWritesCleaningPolicy.EAGER, WriteConcurrencyMode.SINGLE_WRITER, archiveBeyondSavepoint); // min archival commits is 2 and max archival commits is 4. and so, after 5th commit, 3 commits will be archived. for (int i = 1; i < 5; i++) { @@ -502,10 +494,9 @@ public void testSavepointWithArchival(boolean archiveBeyondSavepoint) throws Exc getActiveCommitInstants(Arrays.asList("00000008", "00000009")), commitsAfterArchival); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testMergeSmallArchiveFilesRecoverFromBuildPlanFailed(boolean enableArchiveMerge) throws Exception { - HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, enableArchiveMerge, 3, 209715200); + @Test + public void testCompactionWithCorruptVersionFile() throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, 3); // do ingestion and trigger archive actions here. for (int i = 1; i < 10; i++) { @@ -513,27 +504,16 @@ public void testMergeSmallArchiveFilesRecoverFromBuildPlanFailed(boolean enableA archiveAndGetCommitsList(writeConfig); } - // build a merge small archive plan with dummy content - // this plan can not be deserialized. - HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); - FileStatus[] fsStatuses = metaClient.getFs().globStatus( - new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - List candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList()); - - archiver.reOpenWriter(); - Path plan = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME); - archiver.buildArchiveMergePlan(candidateFiles, plan, ".commits_.archive.3_1-0-1"); - String s = "Dummy Content"; - // stain the current merge plan file. - FileIOUtils.createFileInPath(metaClient.getFs(), plan, Option.of(s.getBytes())); - - // check that damaged plan file will not block archived timeline loading. + // create a version pointer file with invalid version number. + metaClient.getFs().delete(LSMTimeline.getVersionFilePath(metaClient)); + FileIOUtils.createFileInPath(metaClient.getFs(), LSMTimeline.getVersionFilePath(metaClient), Option.of("invalid_version".getBytes(StandardCharsets.UTF_8))); + + // check that invalid manifest file will not block archived timeline loading. HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); - assertEquals(9 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); + assertEquals(5 * 3 + 4, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); - // trigger several archive after left damaged merge small archive file plan. + // trigger several archive with the invalid manifest file. for (int i = 1; i < 10; i++) { testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); archiveAndGetCommitsList(writeConfig); @@ -544,19 +524,18 @@ public void testMergeSmallArchiveFilesRecoverFromBuildPlanFailed(boolean enableA HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload(); // check instant number - assertEquals(18 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants()); + assertEquals(4 * 3 + 14, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants()); - // if there are damaged archive files and damaged plan, hoodie need throw ioe while loading archived timeline. - Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1"); - FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes())); + // if there are damaged archive files and damaged plan, hoodie can still load correctly. + Path damagedFile = new Path(metaClient.getArchivePath(), "300_301_1.parquet"); + FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of("dummy".getBytes())); - assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload()); + assertDoesNotThrow(() -> metaClient.getArchivedTimeline().reload(), "Archived timeline can skip the invalid data and manifest files smartly"); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testMergeSmallArchiveFilesRecoverFromMergeFailed(boolean enableArchiveMerge) throws Exception { - HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, enableArchiveMerge, 3, 209715200); + @Test + public void testCompactionRecoverWithoutManifestFile() throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, 3); // do ingestion and trigger archive actions here. for (int i = 1; i < 10; i++) { @@ -566,122 +545,62 @@ public void testMergeSmallArchiveFilesRecoverFromMergeFailed(boolean enableArchi // do a single merge small archive files HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); - FileStatus[] fsStatuses = metaClient.getFs().globStatus( - new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - List candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList()); - archiver.reOpenWriter(); + LSMTimelineWriter timelineWriter = LSMTimelineWriter.getInstance(writeConfig, table); + List candidateFiles = LSMTimeline.latestSnapshotManifest(metaClient).getFiles().stream() + .sorted().map(HoodieLSMTimelineManifest.LSMFileEntry::getFileName).collect(Collectors.toList()); - archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1"); - archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList())); - HoodieLogFormat.Writer writer = archiver.reOpenWriter(); + String compactedFileName = LSMTimelineWriter.compactedFileName(candidateFiles); + timelineWriter.compactFiles(candidateFiles, compactedFileName); // check loading archived and active timeline success HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); - assertEquals(9 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.reload().countInstants()); - - String s = "Dummy Content"; - // stain the current merged archive file. - FileIOUtils.createFileInPath(metaClient.getFs(), writer.getLogFile().getPath(), Option.of(s.getBytes())); - - // do another archive actions with merge small archive files. - for (int i = 1; i < 10; i++) { - testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); - archiveAndGetCommitsList(writeConfig); - } - - // check result. - // we need to load archived timeline successfully and ignore the parsing damage merged archive files exception. - HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false); - HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload(); - - assertEquals(18 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants()); - - // if there are a damaged merged archive files and other common damaged archive file. - // hoodie need throw ioe while loading archived timeline because of parsing the damaged archive file. - Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1"); - FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes())); - - assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload()); + assertEquals(5 * 3 + 4, rawActiveTimeline.countInstants() + archivedTimeLine.reload().countInstants()); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testMergeSmallArchiveFilesRecoverFromDeleteFailed(boolean enableArchiveMerge) throws Exception { - HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, enableArchiveMerge, 3, 209715200); + @Test + public void testCompactionCleaning() throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, 3); // do ingestion and trigger archive actions here. - for (int i = 1; i < 10; i++) { + for (int i = 1; i < 19; i++) { testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); archiveAndGetCommitsList(writeConfig); } - - // do a single merge small archive files - HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient); - HoodieTimelineArchiver archiver = new HoodieTimelineArchiver(writeConfig, table); - FileStatus[] fsStatuses = metaClient.getFs().globStatus( - new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - List candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList()); - - archiver.reOpenWriter(); - - archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1"); - archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList())); - archiver.reOpenWriter(); - - // delete only one of the small archive file to simulate delete action failed. - metaClient.getFs().delete(fsStatuses[0].getPath()); + // now we have version 6, 7, 8, 9 version of snapshots // loading archived timeline and active timeline success HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); - assertEquals(9 * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); - - // do another archive actions with merge small archive files. - for (int i = 1; i < 10; i++) { - testTable.doWriteOperation("1000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); - archiveAndGetCommitsList(writeConfig); - } + HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline(); + assertEquals(4 * 3 + 14, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); - // check result. - HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false); - HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline().reload(); - - assertEquals(18 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants()); + assertEquals(9, LSMTimeline.latestSnapshotVersion(metaClient)); + assertEquals(Arrays.asList(7, 8, 9), LSMTimeline.allSnapshotVersions(metaClient).stream().sorted().collect(Collectors.toList())); } - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testLoadArchiveTimelineWithDamagedPlanFile(boolean enableArchiveMerge) throws Exception { - HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 2, enableArchiveMerge, 3, 209715200); + @Test + public void testReadArchivedCompactionPlan() throws Exception { + HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 4, 5, 5, HoodieTableType.MERGE_ON_READ); // do ingestion and trigger archive actions here. - int numInstant = 12; - for (int i = 1; i < 12; i++) { - testTable.doWriteOperation( - "000000" + String.format("%02d", i), - WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), - Arrays.asList("p1", "p2"), - 2); + for (int i = 1; i < 11; i += 2) { + testTable.doWriteOperation(String.format("%08d", i), WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2); + testTable.doCompaction(String.format("%08d", (i + 1)), Arrays.asList("p1", "p2")); archiveAndGetCommitsList(writeConfig); } - Path plan = new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME); - String s = "Dummy Content"; - // stain the current merge plan file. - FileIOUtils.createFileInPath(metaClient.getFs(), plan, Option.of(s.getBytes())); - - // check that damaged plan file will not block archived timeline loading. - HoodieActiveTimeline rawActiveTimeline = new HoodieActiveTimeline(metaClient, false); - HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline().reload(); - assertEquals((numInstant - 1) * 3, rawActiveTimeline.countInstants() + archivedTimeLine.countInstants()); - - // if there are damaged archive files and damaged plan, hoodie need throw ioe while loading archived timeline. - Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1"); - FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes())); - - assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload()); + // loading archived timeline instants + HoodieArchivedTimeline archivedTimeLine = metaClient.getArchivedTimeline(); + // load instant details + archivedTimeLine.loadCompactionDetailsInMemory("00000001", "00000011"); + List compactionInstants = archivedTimeLine.getCommitTimeline().getInstants(); + assertEquals(2, compactionInstants.size(), "Two compactions instants should be archived."); + List> planDetailsList = compactionInstants.stream().map(archivedTimeLine::getInstantDetails).collect(Collectors.toList()); + assertTrue(planDetailsList.stream().allMatch(Option::isPresent), "All the compaction instants should have plan details."); + // parse the compaction plan for each instant + for (Option planDetails : planDetailsList) { + assertDoesNotThrow(() -> TimelineMetadataUtils.deserializeCompactionPlan(planDetails.get())); + } } @Test @@ -696,7 +615,7 @@ public void testArchivalWithMultiWriters() throws Exception { private void testArchivalWithMultiWriters(boolean enableMetadata) throws Exception { HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 4, 5, 5, 2, - HoodieTableType.COPY_ON_WRITE, false, 10, 209715200, + HoodieTableType.COPY_ON_WRITE, 10, HoodieFailedWritesCleaningPolicy.LAZY, WriteConcurrencyMode.OPTIMISTIC_CONCURRENCY_CONTROL); final ExecutorService executors = Executors.newFixedThreadPool(2); @@ -765,45 +684,6 @@ public static CompletableFuture allOfTerminateOnFailure(List candidateFiles = Arrays.stream(fsStatuses).map(fs -> fs.getPath().toString()).collect(Collectors.toList()); - - archiver.reOpenWriter(); - - archiver.buildArchiveMergePlan(candidateFiles, new Path(metaClient.getArchivePath(), HoodieArchivedTimeline.MERGE_ARCHIVE_PLAN_NAME), ".commits_.archive.3_1-0-1"); - archiver.mergeArchiveFiles(Arrays.stream(fsStatuses).collect(Collectors.toList())); - HoodieLogFormat.Writer writer = archiver.reOpenWriter(); - - String s = "Dummy Content"; - // stain the current merged archive file. - FileIOUtils.createFileInPath(metaClient.getFs(), writer.getLogFile().getPath(), Option.of(s.getBytes())); - - // if there's only a damaged merged archive file, we need to ignore the exception while reading this damaged file. - HoodieActiveTimeline rawActiveTimeline1 = new HoodieActiveTimeline(metaClient, false); - HoodieArchivedTimeline archivedTimeLine1 = metaClient.getArchivedTimeline(); - - assertEquals(9 * 3, archivedTimeLine1.countInstants() + rawActiveTimeline1.countInstants()); - - // if there are a damaged merged archive files and other common damaged archive file. - // hoodie need throw ioe while loading archived timeline because of parsing the damaged archive file. - Path damagedFile = new Path(metaClient.getArchivePath(), ".commits_.archive.300_1-0-1"); - FileIOUtils.createFileInPath(metaClient.getFs(), damagedFile, Option.of(s.getBytes())); - - assertThrows(HoodieException.class, () -> metaClient.getArchivedTimeline().reload()); - } - @ParameterizedTest @ValueSource(booleans = {true, false}) public void testNoArchivalUntilMaxArchiveConfigWithExtraInflightCommits(boolean enableMetadata) throws Exception { @@ -1013,7 +893,6 @@ public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) List commitsAfterArchival = commitsList.getValue(); List archivedInstants = getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003", "00000004"), HoodieTimeline.DELTA_COMMIT_ACTION); - archivedInstants.add(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "00000002")); archivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000002")); verifyArchival(archivedInstants, getActiveCommitInstants(Arrays.asList("00000005", "00000006", "00000007", "00000008"), HoodieTimeline.DELTA_COMMIT_ACTION), @@ -1138,7 +1017,7 @@ public void testArchiveTableWithCleanCommits(boolean enableMetadata) throws Exce } else { if (i == 8) { // when i == 7 compaction in metadata table will be triggered - // and afterwards archival in datatable will kick in when i == 8. + // and after wards archival in datatable will kick in when i == 8. // 1,2,3,4,5,6,7,8 : after archival -> 1,4,5,6,7,8 (bcoz, 2,3,4,5 and 6 are clean commits and are eligible for archival) List expectedActiveInstants = new ArrayList<>(); expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001", "00000008"))); @@ -1210,15 +1089,11 @@ public void testArchiveCompletedRollbackAndClean(boolean isEmpty, boolean enable List expectedArchivedInstants = new ArrayList<>(); for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant++) { createCleanMetadata(startInstant + "", false, false, isEmpty || i % 2 == 0); - expectedArchivedInstants.add(new HoodieInstant(State.REQUESTED, HoodieTimeline.CLEAN_ACTION, startInstant + "")); - expectedArchivedInstants.add(new HoodieInstant(State.INFLIGHT, HoodieTimeline.CLEAN_ACTION, startInstant + "")); expectedArchivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.CLEAN_ACTION, startInstant + "")); } for (int i = 0; i < maxInstantsToKeep + 1; i++, startInstant += 2) { createCommitAndRollbackFile(startInstant + 1 + "", startInstant + "", false, isEmpty || i % 2 == 0); - expectedArchivedInstants.add(new HoodieInstant(State.REQUESTED, HoodieTimeline.ROLLBACK_ACTION, startInstant + "")); - expectedArchivedInstants.add(new HoodieInstant(State.INFLIGHT, HoodieTimeline.ROLLBACK_ACTION, startInstant + "")); expectedArchivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.ROLLBACK_ACTION, startInstant + "")); } @@ -1772,24 +1647,13 @@ private void verifyArchival(List expectedArchivedInstants, List getArchivedInstants(HoodieInstant instant) { - List instants = new ArrayList<>(); - if (instant.getAction().equals(HoodieTimeline.COMMIT_ACTION) || instant.getAction().equals(HoodieTimeline.DELTA_COMMIT_ACTION) - || instant.getAction().equals(HoodieTimeline.CLEAN_ACTION) || instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { - instants.add(new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp())); - } - instants.add(new HoodieInstant(State.INFLIGHT, instant.getAction(), instant.getTimestamp())); - instants.add(new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp())); - return instants; - } - private List getAllArchivedCommitInstants(List commitTimes) { return getAllArchivedCommitInstants(commitTimes, HoodieTimeline.COMMIT_ACTION); } private List getAllArchivedCommitInstants(List commitTimes, String action) { List allInstants = new ArrayList<>(); - commitTimes.forEach(entry -> allInstants.addAll(getArchivedInstants(new HoodieInstant(State.COMPLETED, action, entry)))); + commitTimes.forEach(commitTime -> allInstants.add(new HoodieInstant(State.COMPLETED, action, commitTime))); return allInstants; } 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 c2aceae0b524..17d8adbe25cb 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 @@ -24,11 +24,11 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieRequestedReplaceMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.SparkRDDReadClient; import org.apache.hudi.client.SparkRDDWriteClient; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.HoodieCleanStat; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; diff --git a/hudi-common/src/main/avro/HoodieLSMTimelineInstant.avsc b/hudi-common/src/main/avro/HoodieLSMTimelineInstant.avsc new file mode 100644 index 000000000000..1c523c532a09 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieLSMTimelineInstant.avsc @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +{ + "type":"record", + "name":"HoodieLSMTimelineInstant", + "namespace":"org.apache.hudi.avro.model", + "fields":[ + { + "name":"instantTime", + "type":["null","string"], + "default": null + }, + { + "name":"completionTime", + "type":["null","string"], + "default": null + }, + { + "name":"action", + "type":["null","string"], + "default": null + }, + { + "name":"metadata", + "type":["null", "bytes"], + "default": null + }, + { + "name":"plan", + "type":["null", "bytes"], + "default": null + }, + { + "name":"version", + "type":"int", + "default": 1 + } + ] +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java index 71825a2fd347..ffc523463d54 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/fs/FSUtils.java @@ -837,6 +837,35 @@ public static List getFileStatusAtLevel( return result; } + public static Map deleteFilesParallelize( + HoodieTableMetaClient metaClient, + List paths, + HoodieEngineContext context, + int parallelism, + boolean ignoreFailed) { + return FSUtils.parallelizeFilesProcess(context, + metaClient.getFs(), + parallelism, + pairOfSubPathAndConf -> { + Path file = new Path(pairOfSubPathAndConf.getKey()); + try { + FileSystem fs = metaClient.getFs(); + if (fs.exists(file)) { + return fs.delete(file, false); + } + return true; + } catch (IOException e) { + if (!ignoreFailed) { + throw new HoodieIOException("Failed to delete : " + file, e); + } else { + LOG.warn("Ignore failed deleting : " + file); + return true; + } + } + }, + paths); + } + /** * Serializable function interface. * diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLSMTimelineManifest.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLSMTimelineManifest.java new file mode 100644 index 000000000000..e5b36d7ce187 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieLSMTimelineManifest.java @@ -0,0 +1,131 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.apache.hudi.common.util.JsonUtils; + +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.Serializable; +import java.util.ArrayList; +import java.util.List; +import java.util.stream.Collectors; + +/** + * Manifest entry for a version snapshot of the archived timeline. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieLSMTimelineManifest implements Serializable { + private static final Logger LOG = LoggerFactory.getLogger(HoodieLSMTimelineManifest.class); + + public static final HoodieLSMTimelineManifest EMPTY = new HoodieLSMTimelineManifest(); + + private final List files; + + // for ser/deser + public HoodieLSMTimelineManifest() { + this.files = new ArrayList<>(); + } + + public HoodieLSMTimelineManifest(List files) { + this.files = files; + } + + public void addFile(String fileName, long fileLen) { + this.files.add(LSMFileEntry.getInstance(fileName, fileLen)); + } + + public void addFile(LSMFileEntry fileEntry) { + this.files.add(fileEntry); + } + + public List getFiles() { + return files; + } + + public List getFileNames() { + return files.stream().map(LSMFileEntry::getFileName).collect(Collectors.toList()); + } + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + + public String toJsonString() throws IOException { + return JsonUtils.getObjectMapper().writerWithDefaultPrettyPrinter().writeValueAsString(this); + } + + public static T fromJsonString(String jsonStr, Class clazz) throws Exception { + if (jsonStr == null || jsonStr.isEmpty()) { + // For empty commit file (no data or something bad happen). + return clazz.newInstance(); + } + return JsonUtils.getObjectMapper().readValue(jsonStr, clazz); + } + + public HoodieLSMTimelineManifest copy(List filesToRemove) { + if (filesToRemove.isEmpty()) { + return new HoodieLSMTimelineManifest(new ArrayList<>(this.files)); + } + List newFiles = this.files.stream().filter(fileEntry -> !filesToRemove.contains(fileEntry.getFileName())).collect(Collectors.toList()); + return new HoodieLSMTimelineManifest(newFiles); + } + + // ------------------------------------------------------------------------- + // Inner Class + // ------------------------------------------------------------------------- + + /** + * A file entry. + */ + public static class LSMFileEntry implements Serializable, Comparable { + private String fileName; + private long fileLen; + + // for ser/deser + public LSMFileEntry() { + } + + private LSMFileEntry(String fileName, long fileLen) { + this.fileName = fileName; + this.fileLen = fileLen; + } + + public static LSMFileEntry getInstance(String fileName, long fileLen) { + return new LSMFileEntry(fileName, fileLen); + } + + public String getFileName() { + return fileName; + } + + public long getFileLen() { + return fileLen; + } + + @Override + public int compareTo(LSMFileEntry other) { + // sorts the files by order of min instant time in file name. + return this.fileName.compareTo(other.fileName); + } + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java index 589f1e6cfbf7..1b29ba8f46f0 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/HoodieTableMetaClient.java @@ -982,7 +982,7 @@ public PropertyBuilder set(Map props) { public PropertyBuilder fromMetaClient(HoodieTableMetaClient metaClient) { return setTableType(metaClient.getTableType()) .setTableName(metaClient.getTableConfig().getTableName()) - .setArchiveLogFolder(metaClient.getArchivePath()) + .setArchiveLogFolder(metaClient.getTableConfig().getArchivelogFolder()) .setPayloadClassName(metaClient.getTableConfig().getPayloadClass()) .setRecordMergerStrategy(metaClient.getTableConfig().getRecordMergerStrategy()); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java index 1a36bb15d570..978f61f4caf8 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieActiveTimeline.java @@ -366,7 +366,7 @@ public Option readRollbackInfoAsBytes(HoodieInstant instant) { public Option readRestoreInfoAsBytes(HoodieInstant instant) { // Rollback metadata are always stored only in timeline .hoodie - return readDataFromPath(new Path(metaClient.getMetaPath(), instant.getFileName())); + return readDataFromPath(getInstantFileNamePath(instant.getFileName())); } //----------------------------------------------------------------- diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java index eb4dc631ed60..fff8e3578298 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieArchivedTimeline.java @@ -18,75 +18,54 @@ package org.apache.hudi.common.table.timeline; -import org.apache.hudi.avro.HoodieAvroUtils; -import org.apache.hudi.avro.model.HoodieArchivedMetaEntry; -import org.apache.hudi.avro.model.HoodieMergeArchiveFilePlan; -import org.apache.hudi.common.fs.HoodieWrapperFileSystem; -import org.apache.hudi.common.model.HoodieLogFile; -import org.apache.hudi.common.model.HoodiePartitionMetadata; -import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.avro.model.HoodieLSMTimelineInstant; import org.apache.hudi.common.model.HoodieRecord.HoodieRecordType; import org.apache.hudi.common.table.HoodieTableMetaClient; -import org.apache.hudi.common.table.log.HoodieLogFormat; -import org.apache.hudi.common.table.log.block.HoodieAvroDataBlock; -import org.apache.hudi.common.table.log.block.HoodieLogBlock; -import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.CollectionUtils; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; -import org.apache.hudi.common.util.StringUtils; +import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.exception.HoodieIOException; +import org.apache.hudi.io.storage.HoodieAvroParquetReader; +import org.apache.hudi.io.storage.HoodieFileReaderFactory; +import org.apache.avro.Schema; import org.apache.avro.generic.GenericRecord; import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.annotation.Nonnull; +import javax.annotation.Nullable; import java.io.IOException; import java.io.Serializable; -import java.nio.charset.StandardCharsets; +import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; -import java.util.Comparator; -import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; -import java.util.Spliterator; -import java.util.Spliterators; +import java.util.concurrent.ConcurrentHashMap; import java.util.function.Function; -import java.util.regex.Matcher; -import java.util.regex.Pattern; -import java.util.stream.StreamSupport; /** - * Represents the Archived Timeline for the Hoodie table. Instants for the last 12 hours (configurable) is in the - * ActiveTimeline and the rest are in ArchivedTimeline. - *

- *

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

- *

- * This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized. + * Represents the Archived Timeline for the Hoodie table. + * + *

Timeline Refresh

+ *

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

Serialization/De-serialization

+ *

This class can be serialized and de-serialized and on de-serialization the FileSystem is re-initialized. */ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { - public static final String MERGE_ARCHIVE_PLAN_NAME = "mergeArchivePlan"; - private static final Pattern ARCHIVE_FILE_PATTERN = - Pattern.compile("^\\.commits_\\.archive\\.([0-9]+).*"); - - private static final String HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX = "commits"; - private static final String ACTION_TYPE_KEY = "actionType"; - private static final String ACTION_STATE = "actionState"; - private static final String STATE_TRANSITION_TIME = "stateTransitionTime"; + private static final String INSTANT_TIME_ARCHIVED_META_FIELD = "instantTime"; + private static final String COMPLETION_TIME_ARCHIVED_META_FIELD = "completionTime"; + private static final String ACTION_ARCHIVED_META_FIELD = "action"; + private static final String METADATA_ARCHIVED_META_FIELD = "metadata"; + private static final String PLAN_ARCHIVED_META_FIELD = "plan"; private HoodieTableMetaClient metaClient; - private final Map readCommits = new HashMap<>(); + private final Map readCommits = new ConcurrentHashMap<>(); private static final Logger LOG = LoggerFactory.getLogger(HoodieArchivedTimeline.class); @@ -97,7 +76,7 @@ public class HoodieArchivedTimeline extends HoodieDefaultTimeline { */ public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { this.metaClient = metaClient; - setInstants(this.loadInstants(false)); + setInstants(this.loadInstants()); // multiple casts will make this lambda serializable - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; @@ -109,8 +88,7 @@ public HoodieArchivedTimeline(HoodieTableMetaClient metaClient) { */ public HoodieArchivedTimeline(HoodieTableMetaClient metaClient, String startTs) { this.metaClient = metaClient; - setInstants(loadInstants(new StartTsFilter(startTs), true, - record -> HoodieInstant.State.COMPLETED.toString().equals(record.get(ACTION_STATE).toString()))); + setInstants(loadInstants(new StartTsFilter(startTs), LoadMode.METADATA)); // multiple casts will make this lambda serializable - // http://docs.oracle.com/javase/specs/jls/se8/html/jls-15.html#jls-15.16 this.details = (Function> & Serializable) this::getInstantDetails; @@ -133,21 +111,12 @@ private void readObject(java.io.ObjectInputStream in) throws IOException, ClassN in.defaultReadObject(); } - public static Path getArchiveLogPath(String archiveFolder) { - return new Path(archiveFolder, HOODIE_COMMIT_ARCHIVE_LOG_FILE_PREFIX); - } - public void loadInstantDetailsInMemory(String startTs, String endTs) { loadInstants(startTs, endTs); } public void loadCompletedInstantDetailsInMemory() { - loadInstants(null, true, - record -> { - // Very old archived instants don't have action state set. - Object action = record.get(ACTION_STATE); - return action == null || HoodieInstant.State.COMPLETED.toString().equals(action.toString()); - }); + loadInstants(null, LoadMode.METADATA); } public void loadCompactionDetailsInMemory(String compactionInstantTime) { @@ -156,13 +125,9 @@ public void loadCompactionDetailsInMemory(String compactionInstantTime) { public void loadCompactionDetailsInMemory(String startTs, String endTs) { // load compactionPlan - loadInstants(new TimeRangeFilter(startTs, endTs), true, - record -> { - // Older files don't have action state set. - Object action = record.get(ACTION_STATE); - return record.get(ACTION_TYPE_KEY).toString().equals(HoodieTimeline.COMPACTION_ACTION) - && (action == null || HoodieInstant.State.INFLIGHT.toString().equals(action.toString())); - } + loadInstants(new TimeRangeFilter(startTs, endTs), LoadMode.PLAN, + record -> record.get(ACTION_ARCHIVED_META_FIELD).toString().equals(HoodieTimeline.COMMIT_ACTION) + && record.get(PLAN_ARCHIVED_META_FIELD) != null ); } @@ -184,146 +149,130 @@ public HoodieArchivedTimeline reload() { return new HoodieArchivedTimeline(metaClient); } - private HoodieInstant readCommit(GenericRecord record, boolean loadDetails) { - final String instantTime = record.get(HoodiePartitionMetadata.COMMIT_TIME_KEY).toString(); - final String action = record.get(ACTION_TYPE_KEY).toString(); - final String stateTransitionTime = (String) record.get(STATE_TRANSITION_TIME); - if (loadDetails) { - getMetadataKey(action).map(key -> { - Object actionData = record.get(key); - if (actionData != null) { - if (action.equals(HoodieTimeline.COMPACTION_ACTION)) { - this.readCommits.put(instantTime, HoodieAvroUtils.indexedRecordToBytes((IndexedRecord) actionData)); - } else { - this.readCommits.put(instantTime, actionData.toString().getBytes(StandardCharsets.UTF_8)); - } - } - return null; - }); - } - return new HoodieInstant(HoodieInstant.State.valueOf(record.get(ACTION_STATE).toString()), action, - instantTime, stateTransitionTime); + private HoodieInstant readCommit(GenericRecord record, LoadMode loadMode) { + final String instantTime = record.get(INSTANT_TIME_ARCHIVED_META_FIELD).toString(); + final String action = record.get(ACTION_ARCHIVED_META_FIELD).toString(); + final String completionTime = record.get(COMPLETION_TIME_ARCHIVED_META_FIELD).toString(); + loadInstantDetails(record, instantTime, loadMode); + return new HoodieInstant(HoodieInstant.State.COMPLETED, action, instantTime, completionTime); } - @Nonnull - private Option getMetadataKey(String action) { - switch (action) { - case HoodieTimeline.CLEAN_ACTION: - return Option.of("hoodieCleanMetadata"); - case HoodieTimeline.COMMIT_ACTION: - case HoodieTimeline.DELTA_COMMIT_ACTION: - return Option.of("hoodieCommitMetadata"); - case HoodieTimeline.ROLLBACK_ACTION: - return Option.of("hoodieRollbackMetadata"); - case HoodieTimeline.SAVEPOINT_ACTION: - return Option.of("hoodieSavePointMetadata"); - case HoodieTimeline.COMPACTION_ACTION: - case HoodieTimeline.LOG_COMPACTION_ACTION: - return Option.of("hoodieCompactionPlan"); - case HoodieTimeline.REPLACE_COMMIT_ACTION: - return Option.of("hoodieReplaceCommitMetadata"); - case HoodieTimeline.INDEXING_ACTION: - return Option.of("hoodieIndexCommitMetadata"); + private void loadInstantDetails(GenericRecord record, String instantTime, LoadMode loadMode) { + switch (loadMode) { + case METADATA: + ByteBuffer commitMeta = (ByteBuffer) record.get(METADATA_ARCHIVED_META_FIELD); + if (commitMeta != null) { + // in case the entry comes from an empty completed meta file + this.readCommits.put(instantTime, commitMeta.array()); + } + break; + case PLAN: + ByteBuffer plan = (ByteBuffer) record.get(PLAN_ARCHIVED_META_FIELD); + if (plan != null) { + // in case the entry comes from an empty completed meta file + this.readCommits.put(instantTime, plan.array()); + } + break; default: - LOG.error(String.format("Unknown action in metadata (%s)", action)); - return Option.empty(); + // no operation } } - private List loadInstants(boolean loadInstantDetails) { - return loadInstants(null, loadInstantDetails); + private List loadInstants() { + return loadInstants(null, LoadMode.SLIM); } private List loadInstants(String startTs, String endTs) { - return loadInstants(new TimeRangeFilter(startTs, endTs), true); + return loadInstants(new TimeRangeFilter(startTs, endTs), LoadMode.METADATA); } - private List loadInstants(TimeRangeFilter filter, boolean loadInstantDetails) { - return loadInstants(filter, loadInstantDetails, record -> true); + private List loadInstants(TimeRangeFilter filter, LoadMode loadMode) { + return loadInstants(filter, loadMode, r -> true); } /** * This is method to read selected instants. Do NOT use this directly use one of the helper methods above * If loadInstantDetails is set to true, this would also update 'readCommits' map with commit details * If filter is specified, only the filtered instants are loaded - * If commitsFilter is specified, only the filtered records are loaded + * If commitsFilter is specified, only the filtered records are loaded. */ - private List loadInstants(TimeRangeFilter filter, boolean loadInstantDetails, - Function commitsFilter) { + private List loadInstants( + @Nullable TimeRangeFilter filter, + LoadMode loadMode, + Function commitsFilter) { try { // List all files - FileStatus[] fsStatuses = metaClient.getFs().globStatus( - new Path(metaClient.getArchivePath() + "/.commits_.archive*")); - - // Sort files by version suffix in reverse (implies reverse chronological order) - Arrays.sort(fsStatuses, new ArchiveFileVersionComparator()); - - Set instantsInRange = new HashSet<>(); - for (FileStatus fs : fsStatuses) { - // Read the archived file - try (HoodieLogFormat.Reader reader = HoodieLogFormat.newReader(metaClient.getFs(), - new HoodieLogFile(fs.getPath()), HoodieArchivedMetaEntry.getClassSchema())) { - int instantsInPreviousFile = instantsInRange.size(); - // Read the avro blocks - while (reader.hasNext()) { - HoodieLogBlock block = reader.next(); - if (block instanceof HoodieAvroDataBlock) { - HoodieAvroDataBlock avroBlock = (HoodieAvroDataBlock) block; - // TODO If we can store additional metadata in datablock, we can skip parsing records - // (such as startTime, endTime of records in the block) - try (ClosableIterator> itr = avroBlock.getRecordIterator(HoodieRecordType.AVRO)) { - StreamSupport.stream(Spliterators.spliteratorUnknownSize(itr, Spliterator.IMMUTABLE), true) - // Filter blocks in desired time window - .map(r -> (GenericRecord) r.getData()) - .filter(commitsFilter::apply) - .map(r -> readCommit(r, loadInstantDetails)) - .filter(c -> filter == null || filter.isInRange(c)) - .forEach(instantsInRange::add); - } - } - } - - if (filter != null) { - int instantsInCurrentFile = instantsInRange.size() - instantsInPreviousFile; - if (instantsInPreviousFile > 0 && instantsInCurrentFile == 0) { - // Note that this is an optimization to skip reading unnecessary archived files - // This signals we crossed lower bound of desired time window. - break; - } - } - } catch (Exception originalException) { - // merge small archive files may left uncompleted archive file which will cause exception. - // need to ignore this kind of exception here. - try { - Path planPath = new Path(metaClient.getArchivePath(), MERGE_ARCHIVE_PLAN_NAME); - HoodieWrapperFileSystem fileSystem = metaClient.getFs(); - if (fileSystem.exists(planPath)) { - HoodieMergeArchiveFilePlan plan = TimelineMetadataUtils.deserializeAvroMetadata(FileIOUtils.readDataFromPath(fileSystem, planPath).get(), HoodieMergeArchiveFilePlan.class); - String mergedArchiveFileName = plan.getMergedArchiveFileName(); - if (!StringUtils.isNullOrEmpty(mergedArchiveFileName) && fs.getPath().getName().equalsIgnoreCase(mergedArchiveFileName)) { - LOG.warn("Catch exception because of reading uncompleted merging archive file " + mergedArchiveFileName + ". Ignore it here."); - continue; - } - } - throw originalException; - } catch (Exception e) { - // If anything wrong during parsing merge archive plan, we need to throw the original exception. - // For example corrupted archive file and corrupted plan are both existed. - throw originalException; - } - } - } - - ArrayList result = new ArrayList<>(instantsInRange); + List fileNames = LSMTimeline.latestSnapshotManifest(metaClient).getFileNames(); + + Map instantsInRange = new ConcurrentHashMap<>(); + Schema readSchema = LSMTimeline.getReadSchema(loadMode); + fileNames.stream() + .filter(fileName -> filter == null || LSMTimeline.isFileInRange(filter, fileName)) + .parallel().forEach(fileName -> { + // Read the archived file + try (HoodieAvroParquetReader reader = (HoodieAvroParquetReader) HoodieFileReaderFactory.getReaderFactory(HoodieRecordType.AVRO) + .getFileReader(metaClient.getHadoopConf(), new Path(metaClient.getArchivePath(), fileName))) { + try (ClosableIterator iterator = reader.getIndexedRecordIterator(HoodieLSMTimelineInstant.getClassSchema(), readSchema)) { + while (iterator.hasNext()) { + GenericRecord record = (GenericRecord) iterator.next(); + String instantTime = record.get(INSTANT_TIME_ARCHIVED_META_FIELD).toString(); + if (!instantsInRange.containsKey(instantTime) + && (filter == null || filter.isInRange(instantTime)) + && commitsFilter.apply(record)) { + HoodieInstant instant = readCommit(record, loadMode); + instantsInRange.put(instantTime, instant); + } + } + } + } catch (IOException ioException) { + throw new HoodieIOException("Error open file reader for path: " + new Path(metaClient.getArchivePath(), fileName)); + } + }); + + ArrayList result = new ArrayList<>(instantsInRange.values()); Collections.sort(result); return result; } catch (IOException e) { throw new HoodieIOException( - "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); + "Could not load archived commit timeline from path " + metaClient.getArchivePath(), e); } } - private static class TimeRangeFilter { + @Override + public HoodieDefaultTimeline getWriteTimeline() { + // filter in-memory instants + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); + return new HoodieDefaultTimeline(getInstantsAsStream().filter(i -> + readCommits.containsKey(i.getTimestamp())) + .filter(s -> validActions.contains(s.getAction())), details); + } + + // ------------------------------------------------------------------------- + // Inner Class + // ------------------------------------------------------------------------- + + /** + * Different mode for loading the archived instant metadata. + */ + public enum LoadMode { + /** + * Loads the instantTime, completionTime, action. + */ + SLIM, + /** + * Loads the instantTime, completionTime, action, metadata. + */ + METADATA, + /** + * Loads the instantTime, completionTime, plan. + */ + PLAN + } + + /** + * A time based filter with range (startTs, endTs]. + */ + public static class TimeRangeFilter { private final String startTs; private final String endTs; @@ -332,11 +281,14 @@ public TimeRangeFilter(String startTs, String endTs) { this.endTs = endTs; } - public boolean isInRange(HoodieInstant instant) { - return HoodieTimeline.isInRange(instant.getTimestamp(), this.startTs, this.endTs); + public boolean isInRange(String instantTime) { + return HoodieTimeline.isInRange(instantTime, this.startTs, this.endTs); } } + /** + * A time based filter with range [startTs, +∞). + */ private static class StartTsFilter extends TimeRangeFilter { private final String startTs; @@ -345,42 +297,8 @@ public StartTsFilter(String startTs) { this.startTs = startTs; } - public boolean isInRange(HoodieInstant instant) { - return HoodieTimeline.compareTimestamps(instant.getTimestamp(), GREATER_THAN_OR_EQUALS, startTs); - } - } - - /** - * Sort files by reverse order of version suffix in file name. - */ - public static class ArchiveFileVersionComparator implements Comparator, Serializable { - @Override - public int compare(FileStatus f1, FileStatus f2) { - return Integer.compare(getArchivedFileSuffix(f2), getArchivedFileSuffix(f1)); - } - - private int getArchivedFileSuffix(FileStatus f) { - try { - Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(f.getPath().getName()); - if (fileMatcher.matches()) { - return Integer.parseInt(fileMatcher.group(1)); - } - } catch (NumberFormatException e) { - // log and ignore any format warnings - LOG.warn("error getting suffix for archived file: " + f.getPath()); - } - - // return default value in case of any errors - return 0; + public boolean isInRange(String instantTime) { + return HoodieTimeline.compareTimestamps(instantTime, GREATER_THAN_OR_EQUALS, startTs); } } - - @Override - public HoodieDefaultTimeline getWriteTimeline() { - // filter in-memory instants - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, LOG_COMPACTION_ACTION, REPLACE_COMMIT_ACTION); - return new HoodieDefaultTimeline(getInstantsAsStream().filter(i -> - readCommits.containsKey(i.getTimestamp())) - .filter(s -> validActions.contains(s.getAction())), details); - } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java index 901530b11d6e..787740f6c15e 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieInstant.java @@ -49,7 +49,7 @@ public class HoodieInstant implements Serializable, Comparable { /** * A COMPACTION action eventually becomes COMMIT when completed. So, when grouping instants - * for state transitions, this needs to be taken into account + * for state transitions, this needs to be taken into account. */ private static final Map COMPARABLE_ACTIONS = createComparableActionsMap(); @@ -240,7 +240,7 @@ public String getFileName() { throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } - private static final Map createComparableActionsMap() { + private static Map createComparableActionsMap() { Map comparableMap = new HashMap<>(); comparableMap.put(HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.COMMIT_ACTION); comparableMap.put(HoodieTimeline.LOG_COMPACTION_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/LSMTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/LSMTimeline.java new file mode 100644 index 000000000000..450c41bc56a5 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/LSMTimeline.java @@ -0,0 +1,287 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.common.model.HoodieLSMTimelineManifest; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.util.ArchivedInstantReadSchemas; +import org.apache.hudi.common.util.FileIOUtils; +import org.apache.hudi.common.util.Option; +import org.apache.hudi.exception.HoodieException; + +import org.apache.avro.Schema; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.Arrays; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.stream.Collectors; + +/** + * Represents the LSM Timeline for the Hoodie table. + * + *

After several instants are accumulated as a batch on the active timeline, they would be flushed as a parquet file into the LSM timeline. + * In general the timeline is comprised with parquet files with LSM style file layout. Each new operation to the timeline yields + * a new snapshot version. Theoretically, there could be multiple snapshot versions on the timeline. + * + *

The LSM Timeline Layout

+ * + *
+ *   t111, t112 ... t120 ... ->
+ *     \              /
+ *        \        /
+ *            |
+ *            V
+ *   t111_t120_0.parquet, t101_t110_0.parquet,...  t11_t20_0.parquet    L0
+ *                                  \                    /
+ *                                     \              /
+ *                                            |
+ *                                            V
+ *                                    t11_t100_1.parquet                L1
+ *
+ *      manifest_1, manifest_2, ... manifest_12
+ *                                      |
+ *                                      V
+ *                                  _version_
+ * 
+ * + *

The LSM Tree Compaction

+ * Use the universal compaction strategy, that is: when N(by default 10) number of parquet files exist in the current layer, they are merged and flush as a compacted file in the next layer. + * We have no limit for the layer number, assumes there are 10 instants for each file in L0, there could be 100 instants per file in L1, + * so 3000 instants could be represented as 3 parquets in L2, it is pretty fast if we apply concurrent read. + * + *

The benchmark shows 1000 instants reading cost about 10 ms. + * + *

The Archiver & Reader Snapshot Isolation

+ * + *

In order to make snapshot isolation of the LSM timeline write/read, we add two kinds of metadata files for the LSM tree version management: + *

    + *
  1. Manifest file: Each new file in layer 0 or each compaction would generate a new manifest file, the manifest file records the valid file handles of the latest snapshot;
  2. + *
  3. Version file: A version file is generated right after a new manifest file is formed.
  4. + *
+ * + *

The Reader Workflow

+ *
    + *
  • read the latest version;
  • + *
  • read the manifest file for valid file handles;
  • + *
  • read the data files, probably do a data skipping with the parquet file name max min timestamp.
  • + *
+ * + *

The Legacy Files Cleaning and Read Retention

+ * Only triggers file cleaning after a valid compaction. + * + *

Clean Strategy

+ * Keeps only 3 valid snapshot versions for the reader, that means, a file is kept for at lest 3 archival trigger interval, for default configuration, it is 30 instants time span, + * which is far longer that the LSM timeline loading time. + * + *

Instants TTL

+ * The timeline reader only reads instants of last limited days. We will by default skip the instants from LSM timeline that are generated long time ago. + */ +public class LSMTimeline { + private static final Logger LOG = LoggerFactory.getLogger(LSMTimeline.class); + + public static final int LSM_TIMELINE_INSTANT_VERSION_1 = 1; + + private static final String VERSION_FILE_NAME = "_version_"; // _version_ + private static final String MANIFEST_FILE_PREFIX = "manifest_"; // manifest_[N] + + private static final String TEMP_FILE_SUFFIX = ".tmp"; + + private static final Pattern ARCHIVE_FILE_PATTERN = + Pattern.compile("^(\\d+)_(\\d+)_(\\d)\\.parquet"); + + // ------------------------------------------------------------------------- + // Utilities + // ------------------------------------------------------------------------- + public static Schema getReadSchema(HoodieArchivedTimeline.LoadMode loadMode) { + switch (loadMode) { + case SLIM: + return ArchivedInstantReadSchemas.TIMELINE_LSM_SLIM_READ_SCHEMA; + case METADATA: + return ArchivedInstantReadSchemas.TIMELINE_LSM_READ_SCHEMA_WITH_METADATA; + case PLAN: + return ArchivedInstantReadSchemas.TIMELINE_LSM_READ_SCHEMA_WITH_PLAN; + default: + throw new AssertionError("Unexpected"); + } + } + + /** + * Returns whether the given file is located in the filter. + */ + public static boolean isFileInRange(HoodieArchivedTimeline.TimeRangeFilter filter, String fileName) { + String minInstant = getMinInstantTime(fileName); + String maxInstant = getMaxInstantTime(fileName); + return filter.isInRange(minInstant) || filter.isInRange(maxInstant); + } + + /** + * Returns the latest snapshot version. + */ + public static int latestSnapshotVersion(HoodieTableMetaClient metaClient) throws IOException { + Path versionFilePath = getVersionFilePath(metaClient); + if (metaClient.getFs().exists(versionFilePath)) { + try { + Option content = FileIOUtils.readDataFromPath(metaClient.getFs(), versionFilePath); + if (content.isPresent()) { + return Integer.parseInt(new String(content.get(), StandardCharsets.UTF_8)); + } + } catch (Exception e) { + // fallback to manifest file listing. + LOG.warn("Error reading version file {}", versionFilePath, e); + } + } + return allSnapshotVersions(metaClient).stream().max(Integer::compareTo).orElse(-1); + } + + /** + * Returns all the valid snapshot versions. + */ + public static List allSnapshotVersions(HoodieTableMetaClient metaClient) throws IOException { + return Arrays.stream(metaClient.getFs().listStatus(new Path(metaClient.getArchivePath()), getManifestFilePathFilter())) + .map(fileStatus -> fileStatus.getPath().getName()) + .map(LSMTimeline::getManifestVersion) + .collect(Collectors.toList()); + } + + /** + * Returns the latest snapshot metadata files. + */ + public static HoodieLSMTimelineManifest latestSnapshotManifest(HoodieTableMetaClient metaClient) throws IOException { + int latestVersion = latestSnapshotVersion(metaClient); + return latestSnapshotManifest(metaClient, latestVersion); + } + + /** + * Reads the file list from the manifest file for the latest snapshot. + */ + public static HoodieLSMTimelineManifest latestSnapshotManifest(HoodieTableMetaClient metaClient, int latestVersion) { + if (latestVersion < 0) { + // there is no valid snapshot of the timeline. + return HoodieLSMTimelineManifest.EMPTY; + } + // read and deserialize the valid files. + byte[] content = FileIOUtils.readDataFromPath(metaClient.getFs(), getManifestFilePath(metaClient, latestVersion)).get(); + try { + return HoodieLSMTimelineManifest.fromJsonString(new String(content, StandardCharsets.UTF_8), HoodieLSMTimelineManifest.class); + } catch (Exception e) { + throw new HoodieException("Error deserializing manifest entries", e); + } + } + + /** + * Returns the full manifest file path with given version number. + */ + public static Path getManifestFilePath(HoodieTableMetaClient metaClient, int snapshotVersion) { + return new Path(metaClient.getArchivePath(), MANIFEST_FILE_PREFIX + snapshotVersion); + } + + /** + * Returns the full version file path with given version number. + */ + public static Path getVersionFilePath(HoodieTableMetaClient metaClient) { + return new Path(metaClient.getArchivePath(), VERSION_FILE_NAME); + } + + /** + * List all the parquet manifest files. + */ + public static FileStatus[] listAllManifestFiles(HoodieTableMetaClient metaClient) throws IOException { + return metaClient.getFs().listStatus(new Path(metaClient.getArchivePath()), getManifestFilePathFilter()); + } + + /** + * List all the parquet metadata files. + */ + public static FileStatus[] listAllMetaFiles(HoodieTableMetaClient metaClient) throws IOException { + return metaClient.getFs().globStatus( + new Path(metaClient.getArchivePath() + "/*.parquet")); + } + + /** + * Parse the snapshot version from the manifest file name. + */ + public static int getManifestVersion(String fileName) { + return Integer.parseInt(fileName.split("_")[1]); + } + + /** + * Parse the layer number from the file name. + */ + public static int getFileLayer(String fileName) { + try { + Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(fileName); + if (fileMatcher.matches()) { + return Integer.parseInt(fileMatcher.group(3)); + } + } catch (NumberFormatException e) { + // log and ignore any format warnings + LOG.warn("error getting file layout for archived file: " + fileName); + } + + // return default value in case of any errors + return 0; + } + + /** + * Parse the minimum instant time from the file name. + */ + public static String getMinInstantTime(String fileName) { + Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(fileName); + if (fileMatcher.matches()) { + return fileMatcher.group(1); + } else { + throw new HoodieException("Unexpected archival file name: " + fileName); + } + } + + /** + * Parse the maximum instant time from the file name. + */ + public static String getMaxInstantTime(String fileName) { + Matcher fileMatcher = ARCHIVE_FILE_PATTERN.matcher(fileName); + if (fileMatcher.matches()) { + return fileMatcher.group(2); + } else { + throw new HoodieException("Unexpected archival file name: " + fileName); + } + } + + /** + * Returns whether a file belongs to the specified layer {@code layer} within the LSM layout. + */ + public static boolean isFileFromLayer(String fileName, int layer) { + return getFileLayer(fileName) == layer; + } + + /** + * Returns a path filter for the manifest files. + */ + public static PathFilter getManifestFilePathFilter() { + return path -> path.getName().startsWith(MANIFEST_FILE_PREFIX) && !path.getName().endsWith(TEMP_FILE_SUFFIX); + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/ArchivedInstantReadSchemas.java b/hudi-common/src/main/java/org/apache/hudi/common/util/ArchivedInstantReadSchemas.java new file mode 100644 index 000000000000..0bcc5b976d42 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/ArchivedInstantReadSchemas.java @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.util; + +import org.apache.avro.Schema; + +/** + * Avro schema for different archived instant read cases. + */ +public abstract class ArchivedInstantReadSchemas { + public static final Schema TIMELINE_LSM_SLIM_READ_SCHEMA = new Schema.Parser().parse("{\n" + + " \"type\":\"record\",\n" + + " \"name\":\"HoodieArchivedMetaEntryV2\",\n" + + " \"namespace\":\"org.apache.hudi.avro.model\",\n" + + " \"fields\":[\n" + + " {\n" + + " \"name\":\"instantTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"completionTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"action\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + "}"); + + public static final Schema TIMELINE_LSM_READ_SCHEMA_WITH_METADATA = new Schema.Parser().parse("{\n" + + " \"type\":\"record\",\n" + + " \"name\":\"HoodieArchivedMetaEntryV2\",\n" + + " \"namespace\":\"org.apache.hudi.avro.model\",\n" + + " \"fields\":[\n" + + " {\n" + + " \"name\":\"instantTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"completionTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"action\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"metadata\",\n" + + " \"type\":[\"null\", \"bytes\"],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + "}"); + + public static final Schema TIMELINE_LSM_READ_SCHEMA_WITH_PLAN = new Schema.Parser().parse("{\n" + + " \"type\":\"record\",\n" + + " \"name\":\"HoodieArchivedMetaEntryV2\",\n" + + " \"namespace\":\"org.apache.hudi.avro.model\",\n" + + " \"fields\":[\n" + + " {\n" + + " \"name\":\"instantTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"completionTime\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"action\",\n" + + " \"type\":[\"null\",\"string\"],\n" + + " \"default\": null\n" + + " },\n" + + " {\n" + + " \"name\":\"plan\",\n" + + " \"type\":[\"null\", \"bytes\"],\n" + + " \"default\": null\n" + + " }\n" + + " ]\n" + + "}"); +} diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java index af65bac055c3..b15ce11fd530 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroFileReaderBase.java @@ -18,13 +18,14 @@ package org.apache.hudi.io.storage; -import org.apache.avro.Schema; -import org.apache.avro.generic.IndexedRecord; import org.apache.hudi.common.model.HoodieAvroIndexedRecord; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.util.collection.ClosableIterator; import org.apache.hudi.common.util.collection.CloseableMappingIterator; +import org.apache.avro.Schema; +import org.apache.avro.generic.IndexedRecord; + import java.io.IOException; import static org.apache.hudi.common.util.TypeUtils.unsafeCast; @@ -44,5 +45,5 @@ protected ClosableIterator getIndexedRecordIterator(Schema reader return getIndexedRecordIterator(readerSchema, readerSchema); } - protected abstract ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException; + public abstract ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException; } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java index 3d6533a34291..26a4001039ec 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroHFileReader.java @@ -192,7 +192,7 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { + public ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { if (!Objects.equals(readerSchema, requestedSchema)) { throw new UnsupportedOperationException("Schema projections are not supported in HFile reader"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java index 1420424a58b0..00ba9fc3bb06 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroOrcReader.java @@ -72,7 +72,7 @@ public Set filterRowKeys(Set candidateRowKeys) { } @Override - protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { + public ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) { if (!Objects.equals(readerSchema, requestedSchema)) { throw new UnsupportedOperationException("Schema projections are not supported in HFile reader"); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java index ad4d1f16a60c..3dd070fa0a7f 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieAvroParquetReader.java @@ -95,7 +95,7 @@ protected ClosableIterator getIndexedRecordIterator(Schema schema } @Override - protected ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { + public ClosableIterator getIndexedRecordIterator(Schema readerSchema, Schema requestedSchema) throws IOException { return getIndexedRecordIteratorInternal(readerSchema, Option.of(requestedSchema)); } diff --git a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java index d0f2ef025102..4370d7b9c672 100644 --- a/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java +++ b/hudi-common/src/main/java/org/apache/hudi/io/storage/HoodieFileWriter.java @@ -26,7 +26,7 @@ import java.io.IOException; import java.util.Properties; -public interface HoodieFileWriter { +public interface HoodieFileWriter extends AutoCloseable { boolean canWrite(); void writeWithMetadata(HoodieKey key, HoodieRecord record, Schema schema, Properties props) throws IOException; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieLSMTimelineManifest.java b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieLSMTimelineManifest.java new file mode 100644 index 000000000000..a5210626f42f --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/model/TestHoodieLSMTimelineManifest.java @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hudi.common.model; + +import org.junit.jupiter.api.Test; + +import java.util.Arrays; +import java.util.stream.Collectors; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.is; + +/** + * Test cases for {@link HoodieLSMTimelineManifest}. + */ +public class TestHoodieLSMTimelineManifest { + @Test + void testSerializeDeserialize() throws Exception { + HoodieLSMTimelineManifest.LSMFileEntry entry1 = HoodieLSMTimelineManifest.LSMFileEntry.getInstance("file1", 1024); + HoodieLSMTimelineManifest.LSMFileEntry entry2 = HoodieLSMTimelineManifest.LSMFileEntry.getInstance("file2", 2048); + HoodieLSMTimelineManifest manifest = new HoodieLSMTimelineManifest(Arrays.asList(entry1, entry2)); + String expected = "{\n" + + " \"files\" : [ {\n" + + " \"fileName\" : \"file1\",\n" + + " \"fileLen\" : 1024\n" + + " }, {\n" + + " \"fileName\" : \"file2\",\n" + + " \"fileLen\" : 2048\n" + + " } ]\n" + + "}"; + // serialization + assertThat(manifest.toJsonString(), is(expected)); + // deserialization + HoodieLSMTimelineManifest deserialized = HoodieLSMTimelineManifest.fromJsonString(expected, HoodieLSMTimelineManifest.class); + assertThat(deserialized.getFiles().size(), is(2)); + assertThat(deserialized.getFiles().stream().map(HoodieLSMTimelineManifest.LSMFileEntry::getFileName).collect(Collectors.joining(",")), is("file1,file2")); + assertThat(deserialized.getFiles().stream().map(entry -> String.valueOf(entry.getFileLen())).collect(Collectors.joining(",")), is("1024,2048")); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestLSMTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestLSMTimeline.java new file mode 100644 index 000000000000..743a9f01e264 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestLSMTimeline.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.hudi.common.table.timeline; + +import org.apache.hudi.exception.HoodieException; + +import org.junit.jupiter.api.Test; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.CoreMatchers.is; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** + * Test cases for {@link LSMTimeline}. + */ +public class TestLSMTimeline { + @Test + void testParseMinInstantTime() { + String fileName = "001_002_0.parquet"; + String minInstantTime = LSMTimeline.getMinInstantTime(fileName); + assertThat(minInstantTime, is("001")); + assertThrows(HoodieException.class, () -> LSMTimeline.getMinInstantTime("invalid_file_name.parquet")); + } + + @Test + void testParseMaxInstantTime() { + String fileName = "001_002_0.parquet"; + String maxInstantTime = LSMTimeline.getMaxInstantTime(fileName); + assertThat(maxInstantTime, is("002")); + assertThrows(HoodieException.class, () -> LSMTimeline.getMaxInstantTime("invalid_file_name.parquet")); + } + + @Test + void testParseFileLayer() { + String fileName = "001_002_0.parquet"; + int layer = LSMTimeline.getFileLayer(fileName); + assertThat(layer, is(0)); + assertThat("for invalid file name, returns 0", LSMTimeline.getFileLayer("invalid_file_name.parquet"), is(0)); + } +} diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java index 4ace66779ec6..6109bbf449a3 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/FileCreateUtils.java @@ -278,29 +278,16 @@ public static void createRestoreFile(String basePath, String instantTime, Hoodie createMetaFile(basePath, instantTime, HoodieTimeline.RESTORE_ACTION, serializeRestoreMetadata(hoodieRestoreMetadata).get()); } - private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException { - Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); - Files.createDirectories(parentPath); - Path metaFilePath = parentPath.resolve(instantTime + suffix); - if (Files.notExists(metaFilePath)) { - Files.createFile(metaFilePath); - } - } - public static void createRequestedCompaction(String basePath, String instantTime) throws IOException { - createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_COMPACTION_EXTENSION); } public static void createInflightCompaction(String basePath, String instantTime) throws IOException { - createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); - } - - public static void createPendingInflightCompaction(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_COMPACTION_EXTENSION); } public static void createInflightSavepoint(String basePath, String instantTime) throws IOException { - createAuxiliaryMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_SAVEPOINT_EXTENSION); } public static void createPartitionMetaFile(String basePath, String partitionPath) throws IOException { diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java index b1dfa366dd84..8e59981f09b0 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestTable.java @@ -111,7 +111,6 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCleanFile; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCommit; -import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedDeltaCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createRequestedRollbackFile; @@ -232,6 +231,15 @@ public HoodieTestTable addSavepointCommit(String instantTime, HoodieSavepointMet return this; } + public HoodieCommitMetadata createCommitMetadata(String commitTime, WriteOperationType operationType, + List partitions, int filesPerPartition, boolean bootstrap) { + Map>> partitionToFilesNameLengthMap = getPartitionFiles(partitions, + filesPerPartition); + HoodieTestTableState testTableState = getTestTableStateWithPartitionFileInfo(operationType, + metaClient.getTableType(), commitTime, partitionToFilesNameLengthMap); + return createCommitMetadata(operationType, commitTime, testTableState, bootstrap); + } + public HoodieCommitMetadata createCommitMetadata(WriteOperationType operationType, String commitTime, HoodieTestTableState testTableState) { String actionType = getCommitActionType(operationType, metaClient.getTableType()); @@ -461,7 +469,15 @@ public HoodieSavepointMetadata getSavepointMetadata(String instant, Map fileSlices = new ArrayList<>(); + fileSlices.add(new FileSlice("par1", instantTime, "fg-1")); + fileSlices.add(new FileSlice("par2", instantTime, "fg-2")); + HoodieCompactionPlan compactionPlan = CompactionUtils + .buildFromFileSlices(fileSlices.stream().map(fs -> Pair.of(fs.getPartitionPath(), fs)) + .collect(Collectors.toList()), Option.empty(), Option.empty()); + HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); + metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, + TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); currentInstantTime = instantTime; return this; } @@ -470,7 +486,8 @@ public HoodieTestTable addRequestedCompaction(String instantTime, HoodieCompacti HoodieInstant compactionInstant = new HoodieInstant(HoodieInstant.State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, instantTime); metaClient.getActiveTimeline().saveToCompactionRequested(compactionInstant, TimelineMetadataUtils.serializeCompactionPlan(compactionPlan)); - return addRequestedCompaction(instantTime); + currentInstantTime = instantTime; + return this; } public HoodieTestTable addRequestedCompaction(String instantTime, FileSlice... fileSlices) throws IOException { @@ -495,9 +512,10 @@ public HoodieTestTable addInflightCompaction(String instantTime, HoodieCommitMet } public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception { - createRequestedCompaction(basePath, instantTime); - createInflightCompaction(basePath, instantTime); - return addCommit(instantTime, Option.of(commitMetadata)); + addInflightCompaction(instantTime, commitMetadata); + this.inflightCommits.remove(instantTime); + createCommit(basePath, instantTime, Option.of(commitMetadata)); + return this; } public HoodieTestTable addDeletePartitionCommit(String instantTime, String partition, List fileIds) throws Exception { diff --git a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java index 28e75871b9dd..4c4b5e5f2dd4 100644 --- a/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java +++ b/hudi-flink-datasource/hudi-flink/src/test/java/org/apache/hudi/sink/compact/CompactionCommitTestSink.java @@ -18,8 +18,8 @@ package org.apache.hudi.sink.compact; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.common.HoodieFlinkEngineContext; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.config.HoodieArchivalConfig; import org.apache.hudi.exception.HoodieException; diff --git a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java index beff7d67df50..fc60f0ce0638 100644 --- a/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java +++ b/hudi-spark-datasource/hudi-spark/src/main/java/org/apache/hudi/cli/ArchiveExecutorUtils.java @@ -18,8 +18,8 @@ package org.apache.hudi.cli; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.common.HoodieSparkEngineContext; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieMetadataConfig; import org.apache.hudi.common.engine.HoodieEngineContext; import org.apache.hudi.common.model.HoodieAvroPayload; diff --git a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java index 53305c65b909..940f5db90239 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java +++ b/hudi-spark-datasource/hudi-spark/src/test/java/org/apache/hudi/functional/TestSparkConsistentBucketClustering.java @@ -18,11 +18,11 @@ package org.apache.hudi.functional; -import org.apache.hudi.client.HoodieTimelineArchiver; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.client.clustering.plan.strategy.SparkConsistentBucketClusteringPlanStrategy; import org.apache.hudi.client.clustering.run.strategy.SparkConsistentBucketClusteringExecutionStrategy; import org.apache.hudi.client.clustering.update.strategy.SparkConsistentBucketDuplicateUpdateStrategy; +import org.apache.hudi.client.timeline.HoodieTimelineArchiver; import org.apache.hudi.common.config.HoodieStorageConfig; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; @@ -50,8 +50,8 @@ import org.apache.hudi.table.HoodieSparkTable; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.action.cluster.ClusteringPlanPartitionFilterMode; -import org.apache.hudi.testutils.HoodieSparkClientTestHarness; import org.apache.hudi.testutils.HoodieMergeOnReadTestUtils; +import org.apache.hudi.testutils.HoodieSparkClientTestHarness; import org.apache.hudi.testutils.MetadataMergeWriteStatus; import org.apache.avro.Schema; diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala new file mode 100644 index 000000000000..2a0dd554b06d --- /dev/null +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/spark/sql/execution/benchmark/LSMTimelineReadBenchmark.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hudi.DummyActiveAction +import org.apache.hudi.client.common.HoodieJavaEngineContext +import org.apache.hudi.client.timeline.{ActiveAction, LSMTimelineWriter} +import org.apache.hudi.common.model.{HoodieAvroPayload, HoodieTableType, WriteOperationType} +import org.apache.hudi.common.table.timeline.{HoodieArchivedTimeline, HoodieInstant, LSMTimeline} +import org.apache.hudi.common.testutils.{HoodieTestTable, HoodieTestUtils} +import org.apache.hudi.config.{HoodieIndexConfig, HoodieWriteConfig} +import org.apache.hudi.index.HoodieIndex.IndexType +import org.apache.hudi.table.HoodieJavaTable +import org.apache.spark.hudi.benchmark.{HoodieBenchmark, HoodieBenchmarkBase} + +import java.util +import scala.collection.JavaConverters._ + +object LSMTimelineReadBenchmark extends HoodieBenchmarkBase { + + /** + * Java HotSpot(TM) 64-Bit Server VM 1.8.0_351-b10 on Mac OS X 13.4.1 + * Apple M2 + * pref load archived instants: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative + * ------------------------------------------------------------------------------------------------------------------------ + * read shim instants 18 32 15 0.1 17914.8 1.0X + * read instants with commit metadata 19 25 5 0.1 19403.1 0.9X + */ + private def readArchivedInstantsBenchmark(): Unit = { + withTempDir(f => { + val tableName = "testTable" + val tablePath = new Path(f.getCanonicalPath, tableName).toUri.toString + val metaClient = HoodieTestUtils.init(new Configuration(), tablePath, HoodieTableType.COPY_ON_WRITE, tableName) + + val writeConfig = HoodieWriteConfig.newBuilder().withPath(tablePath) + .withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(IndexType.INMEMORY).build()) + .withMarkersType("DIRECT") + .build() + val engineContext = new HoodieJavaEngineContext(new Configuration()) + val writer = LSMTimelineWriter.getInstance(writeConfig, HoodieJavaTable.create(writeConfig, engineContext).asInstanceOf[HoodieJavaTable[HoodieAvroPayload]]) + + val startTs = System.currentTimeMillis() + val startInstant = startTs + 1 + "" + val commitsNum = 10000000 + val batchSize = 2000 + val instantBuffer = new util.ArrayList[ActiveAction]() + for (i <- 1 to commitsNum) { + val instantTime = startTs + i + "" + val action = if (i % 2 == 0) "delta_commit" else "commit" + val instant = new HoodieInstant(HoodieInstant.State.COMPLETED, action, instantTime, instantTime + 1000) + val metadata = HoodieTestTable.of(metaClient).createCommitMetadata(instantTime, WriteOperationType.INSERT, util.Arrays.asList("par1", "par2"), 10, false).toJsonString.getBytes() + instantBuffer.add(new DummyActiveAction(instant, metadata)) + if (i % batchSize == 0) { + // archive 10 instants each time + writer.write(instantBuffer, org.apache.hudi.common.util.Option.empty(), org.apache.hudi.common.util.Option.empty()) + writer.compactAndClean(engineContext) + instantBuffer.clear() + } + } + + val benchmark = new HoodieBenchmark("pref load archived instants", commitsNum, 3) + benchmark.addCase("read slim instants") { _ => + new HoodieArchivedTimeline(metaClient) + } + benchmark.addCase("read instants with commit metadata") { _ => + new HoodieArchivedTimeline(metaClient, startInstant) + } + benchmark.run() + val totalSize = LSMTimeline.latestSnapshotManifest(metaClient).getFiles.asScala + .map(f => f.getFileLen) + .sum + println("Total file size in bytes: " + totalSize) + }) + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + readArchivedInstantsBenchmark() + } +}