diff --git a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java index d0526b0236026..06e980db3602e 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/AbstractHoodieWriteClient.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -40,7 +41,6 @@ import org.apache.hudi.metrics.HoodieMetrics; import org.apache.hudi.table.HoodieTable; import org.apache.hudi.table.upgrade.UpgradeDowngrade; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -49,6 +49,7 @@ import java.io.IOException; import java.nio.charset.StandardCharsets; import java.text.ParseException; +import java.util.Collections; import java.util.List; import java.util.Map; @@ -91,40 +92,46 @@ public boolean commit(String instantTime, JavaRDD writeStatuses) { } /** + * * Commit changes performed at the given instantTime marker. */ public boolean commit(String instantTime, JavaRDD writeStatuses, - Option> extraMetadata) { - List stats = writeStatuses.map(WriteStatus::getStat).collect(); - return commitStats(instantTime, stats, extraMetadata); - } - - public boolean commitStats(String instantTime, List stats, Option> extraMetadata) { - LOG.info("Committing " + instantTime); + Option> extraMetadata) { HoodieTableMetaClient metaClient = createMetaClient(false); String actionType = metaClient.getCommitActionType(); + return commit(instantTime, writeStatuses, extraMetadata, actionType, Collections.emptyMap()); + } + + /** + * Complete changes performed at the given instantTime marker with specified action. + */ + public boolean commit(String instantTime, JavaRDD writeStatuses, + Option> extraMetadata, String commitActionType, Map> partitionToReplacedFileIds) { + List writeStats = writeStatuses.map(WriteStatus::getStat).collect(); + return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds); + } + + public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + String commitActionType) { + return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap()); + } + + public boolean commitStats(String instantTime, List stats, Option> extraMetadata, + String commitActionType, Map> partitionToReplaceFileIds) { + LOG.info("Committing " + instantTime + " action " + commitActionType); // Create a Hoodie table which encapsulated the commits and files visible HoodieTable table = HoodieTable.create(config, hadoopConf); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); - + HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds, extraMetadata, operationType, config.getSchema(), commitActionType); // Finalize write finalizeWrite(table, instantTime, stats); - // add in extra metadata - if (extraMetadata.isPresent()) { - extraMetadata.get().forEach(metadata::addMetadata); - } - metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, config.getSchema()); - metadata.setOperationType(operationType); - try { - activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), + activeTimeline.saveAsComplete(new HoodieInstant(true, commitActionType, instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); postCommit(table, metadata, instantTime, extraMetadata); - emitCommitMetrics(instantTime, metadata, actionType); + emitCommitMetrics(instantTime, metadata, commitActionType); LOG.info("Committed " + instantTime); } catch (IOException e) { throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java index 142ff330a87a9..7970623eeff52 100644 --- a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java +++ b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteClient.java @@ -320,6 +320,22 @@ public JavaRDD bulkInsertPreppedRecords(JavaRDD> pr return postWrite(result, instantTime, table); } + /** + * Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table. + + * @param records HoodieRecords to insert + * @param instantTime Instant time of the commit + * @return JavaRDD[WriteStatus] - RDD of WriteStatus to inspect errors and counts + */ + public HoodieWriteResult insertOverwrite(JavaRDD> records, final String instantTime) { + HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime); + table.validateInsertSchema(); + setOperationType(WriteOperationType.INSERT_OVERWRITE); + this.asyncCleanerService = AsyncCleanerService.startAsyncCleaningIfEnabled(this, instantTime); + HoodieWriteMetadata result = table.insertOverwrite(jsc, instantTime, records); + return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds()); + } + /** * Deletes a list of {@link HoodieKey}s from the Hoodie table, at the supplied instantTime {@link HoodieKey}s will be * de-duped and non existent keys will be removed before deleting. @@ -576,7 +592,7 @@ public String startCommit() { rollbackPendingCommits(); } String instantTime = HoodieActiveTimeline.createNewInstantTime(); - startCommit(instantTime); + startCommitWithTime(instantTime); return instantTime; } @@ -586,24 +602,39 @@ public String startCommit() { * @param instantTime Instant time to be generated */ public void startCommitWithTime(String instantTime) { + HoodieTableMetaClient metaClient = createMetaClient(true); + startCommitWithTime(instantTime, metaClient.getCommitActionType(), metaClient); + } + + /** + * Completes a new commit time for a write operation (insert/update/delete) with specified action. + */ + public void startCommitWithTime(String instantTime, String actionType) { + HoodieTableMetaClient metaClient = createMetaClient(true); + startCommitWithTime(instantTime, actionType, metaClient); + } + + /** + * Completes a new commit time for a write operation (insert/update/delete) with specified action. + */ + private void startCommitWithTime(String instantTime, String actionType, HoodieTableMetaClient metaClient) { // NOTE : Need to ensure that rollback is done before a new commit is started if (rollbackPending) { // Only rollback inflight commit/delta-commits. Do not touch compaction commits rollbackPendingCommits(); } - startCommit(instantTime); + startCommit(instantTime, actionType, metaClient); } - private void startCommit(String instantTime) { - LOG.info("Generate a new instant time " + instantTime); - HoodieTableMetaClient metaClient = createMetaClient(true); + private void startCommit(String instantTime, String actionType, HoodieTableMetaClient metaClient) { + LOG.info("Generate a new instant time: " + instantTime + " action: " + actionType); // if there are pending compactions, their instantTime must not be greater than that of this instant time metaClient.getActiveTimeline().filterPendingCompactionTimeline().lastInstant().ifPresent(latestPending -> ValidationUtils.checkArgument( HoodieTimeline.compareTimestamps(latestPending.getTimestamp(), HoodieTimeline.LESSER_THAN, instantTime), "Latest pending compaction instant time must be earlier than this instant time. Latest Compaction :" + latestPending + ", Ingesting at " + instantTime)); - metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, metaClient.getCommitActionType(), + metaClient.getActiveTimeline().createNewInstant(new HoodieInstant(State.REQUESTED, actionType, instantTime)); } diff --git a/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java new file mode 100644 index 0000000000000..69421a91d6a70 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/client/HoodieWriteResult.java @@ -0,0 +1,68 @@ +/* + * 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; + +import org.apache.spark.api.java.JavaRDD; + +import java.io.Serializable; +import java.util.Collections; +import java.util.List; +import java.util.Map; + +/** + * Result of a write operation. + */ +public class HoodieWriteResult implements Serializable { + + private JavaRDD writeStatuses; + private Map> partitionToReplaceFileIds; + + public HoodieWriteResult(JavaRDD writeStatuses) { + this(writeStatuses, Collections.emptyMap()); + } + + public HoodieWriteResult(JavaRDD writeStatuses, Map> partitionToReplaceFileIds) { + this.writeStatuses = writeStatuses; + this.partitionToReplaceFileIds = partitionToReplaceFileIds; + } + + public JavaRDD getWriteStatuses() { + return this.writeStatuses; + } + + public void setWriteStatuses(final JavaRDD writeStatuses) { + this.writeStatuses = writeStatuses; + } + + public Map> getPartitionToReplaceFileIds() { + return this.partitionToReplaceFileIds; + } + + public void setPartitionToReplaceFileIds(final Map> partitionToReplaceFileIds) { + this.partitionToReplaceFileIds = partitionToReplaceFileIds; + } + + @Override + public String toString() { + return "HoodieWriteResult{" + + "writeStatuses=" + writeStatuses + + ", partitionToReplaceFileIds=" + partitionToReplaceFileIds + + '}'; + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java index 31ba537fa2ce6..a4bcbbf835efb 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieCopyOnWriteTable.java @@ -46,6 +46,7 @@ import org.apache.hudi.table.action.commit.BulkInsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.DeleteCommitActionExecutor; import org.apache.hudi.table.action.commit.InsertCommitActionExecutor; +import org.apache.hudi.table.action.commit.InsertOverwriteCommitActionExecutor; import org.apache.hudi.table.action.commit.InsertPreppedCommitActionExecutor; import org.apache.hudi.table.action.commit.MergeHelper; import org.apache.hudi.table.action.commit.UpsertCommitActionExecutor; @@ -123,6 +124,12 @@ public HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instan this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute(); } + @Override + public HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime, + JavaRDD> records) { + return new InsertOverwriteCommitActionExecutor<>(jsc, config, this, instantTime, records).execute(); + } + @Override public Option scheduleCompaction(JavaSparkContext jsc, String instantTime, Option> extraMetadata) { throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table"); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java index 71bcb31314a81..5c824a6fc92f3 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTable.java @@ -213,6 +213,12 @@ public abstract HoodieWriteMetadata insertPrepped(JavaSparkContext jsc, String i public abstract HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime, JavaRDD> preppedRecords, Option bulkInsertPartitioner); + /** + * Logically delete all existing records and Insert a batch of new records into Hoodie table at the supplied instantTime. + */ + public abstract HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime, + JavaRDD> records); + public HoodieWriteConfig getConfig() { return config; } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java index 4be00a3a58e97..5ce0aedcd85d6 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/HoodieTimelineArchiveLog.java @@ -18,13 +18,12 @@ package org.apache.hudi.table; -import org.apache.hadoop.conf.Configuration; import com.fasterxml.jackson.databind.DeserializationFeature; import com.fasterxml.jackson.databind.ObjectMapper; 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.apache.hudi.avro.model.HoodieArchivedMetaEntry; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRollbackMetadata; @@ -32,6 +31,7 @@ import org.apache.hudi.common.model.ActionType; import org.apache.hudi.common.model.HoodieArchivedLogFile; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieRollingStatMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.log.HoodieLogFormat; @@ -44,6 +44,7 @@ import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineMetadataUtils; +import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.common.util.CleanerUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; @@ -275,6 +276,11 @@ public void archive(JavaSparkContext jsc, List instants) throws H LOG.info("Wrapper schema " + wrapperSchema.toString()); List records = new ArrayList<>(); for (HoodieInstant hoodieInstant : instants) { + boolean deleteSuccess = deleteReplacedFileGroups(jsc, hoodieInstant); + if (!deleteSuccess) { + // throw error and stop archival if deleting replaced file groups failed. + throw new HoodieCommitException("Unable to delete file(s) for " + hoodieInstant.getFileName()); + } try { deleteAnyLeftOverMarkerFiles(jsc, hoodieInstant); records.add(convertToAvroRecord(commitTimeline, hoodieInstant)); @@ -301,6 +307,29 @@ private void deleteAnyLeftOverMarkerFiles(JavaSparkContext jsc, HoodieInstant in } } + private boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieInstant instant) { + if (!instant.isCompleted() || !HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instant.getAction())) { + // only delete files for completed replace instants + return true; + } + + TableFileSystemView fileSystemView = this.table.getFileSystemView(); + List replacedPartitions = getReplacedPartitions(instant); + return ReplaceArchivalHelper.deleteReplacedFileGroups(jsc, metaClient, fileSystemView, instant, replacedPartitions); + } + + private List getReplacedPartitions(HoodieInstant instant) { + try { + HoodieReplaceCommitMetadata metadata = HoodieReplaceCommitMetadata.fromBytes( + metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieReplaceCommitMetadata.class); + + return new ArrayList<>(metadata.getPartitionToReplaceFileIds().keySet()); + } catch (IOException e) { + throw new HoodieCommitException("Failed to archive because cannot delete replace files", e); + } + } + private void writeToFile(Schema wrapperSchema, List records) throws Exception { if (records.size() > 0) { Map header = new HashMap<>(); @@ -334,6 +363,13 @@ private IndexedRecord convertToAvroRecord(HoodieTimeline commitTimeline, HoodieI archivedMetaWrapper.setActionType(ActionType.commit.name()); break; } + case HoodieTimeline.REPLACE_COMMIT_ACTION: { + HoodieReplaceCommitMetadata replaceCommitMetadata = HoodieReplaceCommitMetadata + .fromBytes(commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieReplaceCommitMetadata.class); + archivedMetaWrapper.setHoodieReplaceCommitMetadata(ReplaceArchivalHelper.convertReplaceCommitMetadata(replaceCommitMetadata)); + archivedMetaWrapper.setActionType(ActionType.replacecommit.name()); + break; + } case HoodieTimeline.ROLLBACK_ACTION: { archivedMetaWrapper.setHoodieRollbackMetadata(TimelineMetadataUtils.deserializeAvroMetadata( commitTimeline.getInstantDetails(hoodieInstant).get(), HoodieRollbackMetadata.class)); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java b/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java new file mode 100644 index 0000000000000..317cf0de16e11 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/ReplaceArchivalHelper.java @@ -0,0 +1,99 @@ +/* + * 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.table; + +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.model.FileSlice; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieRollingStatMetadata; +import org.apache.hudi.common.table.HoodieTableMetaClient; +import org.apache.hudi.common.table.timeline.HoodieInstant; +import org.apache.hudi.common.table.view.TableFileSystemView; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; + +import java.io.IOException; +import java.io.Serializable; +import java.util.List; +import java.util.stream.Stream; + +/** + * Operates on marker files for a given write action (commit, delta commit, compaction). + */ +public class ReplaceArchivalHelper implements Serializable { + + private static final Logger LOG = LogManager.getLogger(ReplaceArchivalHelper.class); + + /** + * Convert json metadata to avro format. + */ + public static org.apache.hudi.avro.model.HoodieReplaceCommitMetadata convertReplaceCommitMetadata( + HoodieReplaceCommitMetadata hoodieReplaceCommitMetadata) { + ObjectMapper mapper = new ObjectMapper(); + // Need this to ignore other public get() methods + mapper.configure(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES, false); + org.apache.hudi.avro.model.HoodieReplaceCommitMetadata avroMetaData = + mapper.convertValue(hoodieReplaceCommitMetadata, org.apache.hudi.avro.model.HoodieReplaceCommitMetadata.class); + + // Do not archive Rolling Stats, cannot set to null since AVRO will throw null pointer + avroMetaData.getExtraMetadata().put(HoodieRollingStatMetadata.ROLLING_STAT_METADATA_KEY, ""); + return avroMetaData; + } + + /** + * Delete all files represented by FileSlices in parallel. Return true if all files are deleted successfully. + */ + public static boolean deleteReplacedFileGroups(JavaSparkContext jsc, HoodieTableMetaClient metaClient, + TableFileSystemView fileSystemView, + HoodieInstant instant, List replacedPartitions) { + + JavaRDD partitions = jsc.parallelize(replacedPartitions, replacedPartitions.size()); + return partitions.map(partition -> { + Stream fileSlices = fileSystemView.getReplacedFileGroupsBeforeOrOn(instant.getTimestamp(), partition) + .flatMap(g -> g.getAllRawFileSlices()); + + return fileSlices.map(slice -> deleteFileSlice(slice, metaClient, instant)).allMatch(x -> x); + }).reduce((x, y) -> x & y); + } + + private static boolean deleteFileSlice(FileSlice fileSlice, HoodieTableMetaClient metaClient, HoodieInstant instant) { + boolean baseFileDeleteSuccess = fileSlice.getBaseFile().map(baseFile -> + deletePath(new Path(baseFile.getPath()), metaClient, instant)).orElse(true); + + boolean logFileSuccess = fileSlice.getLogFiles().map(logFile -> + deletePath(logFile.getPath(), metaClient, instant)).allMatch(x -> x); + return baseFileDeleteSuccess & logFileSuccess; + } + + private static boolean deletePath(Path path, HoodieTableMetaClient metaClient, HoodieInstant instant) { + try { + LOG.info("Deleting " + path + " before archiving " + instant); + metaClient.getFs().delete(path); + return true; + } catch (IOException e) { + LOG.error("unable to delete file groups that are replaced", e); + return false; + } + } + +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java b/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java index c46102a4a8659..03ddba36dcde0 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/HoodieWriteMetadata.java @@ -18,15 +18,16 @@ package org.apache.hudi.table.action; -import java.util.List; import org.apache.hudi.client.WriteStatus; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.util.Option; - import org.apache.spark.api.java.JavaRDD; import java.time.Duration; +import java.util.Collections; +import java.util.List; +import java.util.Map; /** * Contains metadata, write-statuses and latency times corresponding to a commit/delta-commit action. @@ -42,6 +43,7 @@ public class HoodieWriteMetadata { private Option> writeStats = Option.empty(); private Option indexUpdateDuration = Option.empty(); private Option finalizeDuration = Option.empty(); + private Option>> partitionToReplaceFileIds = Option.empty(); public HoodieWriteMetadata() { } @@ -101,4 +103,12 @@ public Option getIndexLookupDuration() { public void setIndexLookupDuration(Duration indexLookupDuration) { this.indexLookupDuration = Option.ofNullable(indexLookupDuration); } + + public Map> getPartitionToReplaceFileIds() { + return partitionToReplaceFileIds.orElse(Collections.emptyMap()); + } + + public void setPartitionToReplaceFileIds(Map> partitionToReplaceFileIds) { + this.partitionToReplaceFileIds = Option.ofNullable(partitionToReplaceFileIds); + } } diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java index 9efacdefadef8..0b276391d6e0a 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/BaseCommitActionExecutor.java @@ -30,6 +30,7 @@ import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieInstant.State; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.exception.HoodieCommitException; @@ -40,7 +41,6 @@ import org.apache.hudi.table.WorkloadStat; import org.apache.hudi.table.action.BaseActionExecutor; import org.apache.hudi.table.action.HoodieWriteMetadata; - import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.Partitioner; @@ -48,19 +48,19 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.storage.StorageLevel; +import scala.Tuple2; import java.io.IOException; import java.io.Serializable; import java.nio.charset.StandardCharsets; import java.time.Duration; import java.time.Instant; +import java.util.Collections; import java.util.Comparator; import java.util.Iterator; import java.util.List; import java.util.Map; -import scala.Tuple2; - public abstract class BaseCommitActionExecutor, R> extends BaseActionExecutor { @@ -140,7 +140,7 @@ void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String insta metadata.setOperationType(operationType); HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - String commitActionType = table.getMetaClient().getCommitActionType(); + String commitActionType = getCommitActionType(); HoodieInstant requested = new HoodieInstant(State.REQUESTED, commitActionType, instantTime); activeTimeline.transitionRequestedToInflight(requested, Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)), @@ -150,7 +150,7 @@ void saveWorkloadProfileMetadataToInflight(WorkloadProfile profile, String insta } } - private Partitioner getPartitioner(WorkloadProfile profile) { + protected Partitioner getPartitioner(WorkloadProfile profile) { if (WriteOperationType.isChangingRecords(operationType)) { return getUpsertPartitioner(profile); } else { @@ -191,6 +191,7 @@ protected void updateIndexAndCommitIfNeeded(JavaRDD writeStatusRDD, (HoodieTable)table); result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now())); result.setWriteStatuses(statuses); + result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses)); commitOnAutoCommit(result); } @@ -203,42 +204,40 @@ protected void commitOnAutoCommit(HoodieWriteMetadata result) { } } + protected String getCommitActionType() { + return table.getMetaClient().getCommitActionType(); + } + protected void commit(Option> extraMetadata, HoodieWriteMetadata result) { commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect()); } - protected void commit(Option> extraMetadata, HoodieWriteMetadata result, List stats) { - String actionType = table.getMetaClient().getCommitActionType(); + protected void commit(Option> extraMetadata, HoodieWriteMetadata result, List writeStats) { + String actionType = getCommitActionType(); LOG.info("Committing " + instantTime + ", action Type " + actionType); - // Create a Hoodie table which encapsulated the commits and files visible - HoodieTable table = HoodieTable.create(config, hadoopConf); - - HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); - HoodieCommitMetadata metadata = new HoodieCommitMetadata(); - result.setCommitted(true); - stats.forEach(stat -> metadata.addWriteStat(stat.getPartitionPath(), stat)); - result.setWriteStats(stats); - + result.setWriteStats(writeStats); // Finalize write - finalizeWrite(instantTime, stats, result); - - // add in extra metadata - if (extraMetadata.isPresent()) { - extraMetadata.get().forEach(metadata::addMetadata); - } - metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit()); - metadata.setOperationType(operationType); + finalizeWrite(instantTime, writeStats, result); try { - activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime), + LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType()); + HoodieActiveTimeline activeTimeline = table.getActiveTimeline(); + HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(), + extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType()); + + activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime), Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); LOG.info("Committed " + instantTime); + result.setCommitMetadata(Option.of(metadata)); } catch (IOException e) { throw new HoodieCommitException("Failed to complete commit " + config.getBasePath() + " at time " + instantTime, e); } - result.setCommitMetadata(Option.of(metadata)); + } + + protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { + return Collections.emptyMap(); } /** diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java new file mode 100644 index 0000000000000..054e8cd10bffa --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwriteCommitActionExecutor.java @@ -0,0 +1,80 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.common.model.HoodieRecord; +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.hudi.table.action.HoodieWriteMetadata; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.Partitioner; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import scala.Tuple2; + +import java.util.List; +import java.util.Map; +import java.util.stream.Collectors; + +public class InsertOverwriteCommitActionExecutor> + extends CommitActionExecutor { + + private static final Logger LOG = LogManager.getLogger(InsertOverwriteCommitActionExecutor.class); + + private final JavaRDD> inputRecordsRDD; + + public InsertOverwriteCommitActionExecutor(JavaSparkContext jsc, + HoodieWriteConfig config, HoodieTable table, + String instantTime, JavaRDD> inputRecordsRDD) { + super(jsc, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE); + this.inputRecordsRDD = inputRecordsRDD; + } + + @Override + public HoodieWriteMetadata execute() { + return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable) table, + config.shouldCombineBeforeInsert(), config.getInsertShuffleParallelism(), this, false); + } + + @Override + protected Partitioner getPartitioner(WorkloadProfile profile) { + return new InsertOverwritePartitioner<>(profile, jsc, table, config); + } + + @Override + protected String getCommitActionType() { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } + + protected Map> getPartitionToReplacedFileIds(JavaRDD writeStatuses) { + return writeStatuses.map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath -> + new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap(); + } + + private List getAllExistingFileIds(String partitionPath) { + // because new commit is not complete. it is safe to mark all existing file Ids as old files + return table.getSliceView().getLatestFileSlices(partitionPath).map(fg -> fg.getFileId()).distinct().collect(Collectors.toList()); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java new file mode 100644 index 0000000000000..80db612777583 --- /dev/null +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/commit/InsertOverwritePartitioner.java @@ -0,0 +1,51 @@ +/* + * 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.table.action.commit; + +import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.config.HoodieWriteConfig; +import org.apache.hudi.table.HoodieTable; +import org.apache.hudi.table.WorkloadProfile; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; +import org.apache.spark.api.java.JavaSparkContext; + +import java.util.Collections; +import java.util.List; + +/** + * Packs incoming records to be inserted into buckets (1 bucket = 1 RDD partition). + */ +public class InsertOverwritePartitioner> extends UpsertPartitioner { + + private static final Logger LOG = LogManager.getLogger(InsertOverwritePartitioner.class); + + public InsertOverwritePartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable table, + HoodieWriteConfig config) { + super(profile, jsc, table, config); + } + + /** + * Returns a list of small files in the given partition path. + */ + protected List getSmallFiles(String partitionPath) { + // for overwrite, we ignore all existing files. So dont consider any file to be smallFiles + return Collections.emptyList(); + } +} diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java index 7491f4c08a883..0b4e4c4d297c8 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/BaseRestoreActionExecutor.java @@ -76,7 +76,7 @@ public HoodieRestoreMetadata execute() { try { return finishRestore(instantToMetadata, - instantsToRollback.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), + instantsToRollback, restoreTimer.endTimer() ); } catch (IOException io) { @@ -87,7 +87,7 @@ public HoodieRestoreMetadata execute() { protected abstract HoodieRollbackMetadata rollbackInstant(HoodieInstant rollbackInstant); private HoodieRestoreMetadata finishRestore(Map> instantToMetadata, - List instantsRolledBack, + List instantsRolledBack, long durationInMs) throws IOException { HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.convertRestoreMetadata( diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java index 2c9c2bcc5aaf6..cbd3127596396 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/CopyOnWriteRestoreActionExecutor.java @@ -50,7 +50,8 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback true, true, false); - if (!instantToRollback.getAction().equals(HoodieTimeline.COMMIT_ACTION)) { + if (!instantToRollback.getAction().equals(HoodieTimeline.COMMIT_ACTION) + && !instantToRollback.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { throw new HoodieRollbackException("Unsupported action in rollback instant:" + instantToRollback); } return rollbackActionExecutor.execute(); diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java index 827d8e239a792..edb9acdcc59f7 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/restore/MergeOnReadRestoreActionExecutor.java @@ -54,6 +54,7 @@ protected HoodieRollbackMetadata rollbackInstant(HoodieInstant instantToRollback case HoodieTimeline.COMMIT_ACTION: case HoodieTimeline.DELTA_COMMIT_ACTION: case HoodieTimeline.COMPACTION_ACTION: + case HoodieTimeline.REPLACE_COMMIT_ACTION: // TODO : Get file status and create a rollback stat and file // TODO : Delete the .aux files along with the instant file, okay for now since the archival process will // delete these files when it does not see a corresponding instant file under .hoodie diff --git a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java index ce2c8ca100644..36199c5b55e51 100644 --- a/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java +++ b/hudi-client/src/main/java/org/apache/hudi/table/action/rollback/BaseRollbackActionExecutor.java @@ -109,7 +109,7 @@ public HoodieRollbackMetadata execute() { HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.convertRollbackMetadata( instantTime, Option.of(rollbackTimer.endTimer()), - Collections.singletonList(instantToRollback.getTimestamp()), + Collections.singletonList(instantToRollback), stats); if (!skipTimelinePublish) { finishRollback(rollbackMetadata); diff --git a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java index c9a03c74d6ba5..47edbbd405503 100644 --- a/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java +++ b/hudi-client/src/test/java/org/apache/hudi/client/TestHoodieClientOnCopyOnWriteStorage.java @@ -18,6 +18,9 @@ package org.apache.hudi.client; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.fs.ConsistencyGuardConfig; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieBaseFile; @@ -54,10 +57,6 @@ import org.apache.hudi.testutils.HoodieClientTestBase; import org.apache.hudi.testutils.HoodieClientTestUtils; import org.apache.hudi.testutils.HoodieWriteableTestTable; - -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FSDataInputStream; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -888,6 +887,87 @@ public void testDeletesWithDeleteApi() throws Exception { testDeletes(client, updateBatch3.getRight(), 10, file1, "007", 140, keysSoFar); } + /** + * Test scenario of writing more file groups than existing number of file groups in partition. + */ + @Test + public void testInsertOverwritePartitionHandlingWithMoreRecords() throws Exception { + verifyInsertOverwritePartitionHandling(1000, 3000); + } + + /** + * Test scenario of writing fewer file groups than existing number of file groups in partition. + */ + @Test + public void testInsertOverwritePartitionHandlingWithFewerRecords() throws Exception { + verifyInsertOverwritePartitionHandling(3000, 1000); + } + + /** + * Test scenario of writing similar number file groups in partition. + */ + @Test + public void testInsertOverwritePartitionHandlinWithSimilarNumberOfRecords() throws Exception { + verifyInsertOverwritePartitionHandling(3000, 3000); + } + + /** + * 1) Do write1 (upsert) with 'batch1RecordsCount' number of records. + * 2) Do write2 (insert overwrite) with 'batch2RecordsCount' number of records. + * + * Verify that all records in step1 are overwritten + */ + private void verifyInsertOverwritePartitionHandling(int batch1RecordsCount, int batch2RecordsCount) throws Exception { + final String testPartitionPath = "americas"; + HoodieWriteConfig config = getSmallInsertWriteConfig(2000, false); + HoodieWriteClient client = getHoodieWriteClient(config, false); + dataGen = new HoodieTestDataGenerator(new String[] {testPartitionPath}); + + // Do Inserts + String commitTime1 = "001"; + client.startCommitWithTime(commitTime1); + List inserts1 = dataGen.generateInserts(commitTime1, batch1RecordsCount); + JavaRDD insertRecordsRDD1 = jsc.parallelize(inserts1, 2); + List statuses = client.upsert(insertRecordsRDD1, commitTime1).collect(); + assertNoWriteErrors(statuses); + Set batch1Buckets = statuses.stream().map(s -> s.getFileId()).collect(Collectors.toSet()); + verifyRecordsWritten(commitTime1, inserts1, statuses); + + // Do Insert Overwrite + String commitTime2 = "002"; + client.startCommitWithTime(commitTime2, HoodieTimeline.REPLACE_COMMIT_ACTION); + List inserts2 = dataGen.generateInserts(commitTime2, batch2RecordsCount); + List insertsAndUpdates2 = new ArrayList<>(); + insertsAndUpdates2.addAll(inserts2); + JavaRDD insertAndUpdatesRDD2 = jsc.parallelize(insertsAndUpdates2, 2); + HoodieWriteResult writeResult = client.insertOverwrite(insertAndUpdatesRDD2, commitTime2); + statuses = writeResult.getWriteStatuses().collect(); + assertNoWriteErrors(statuses); + + assertEquals(batch1Buckets, new HashSet<>(writeResult.getPartitionToReplaceFileIds().get(testPartitionPath))); + verifyRecordsWritten(commitTime2, inserts2, statuses); + } + + /** + * Verify data in parquet files matches expected records and commit time. + */ + private void verifyRecordsWritten(String commitTime, List expectedRecords, List allStatus) { + List records = new ArrayList<>(); + for (WriteStatus status : allStatus) { + Path filePath = new Path(basePath, status.getStat().getPath()); + records.addAll(ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), filePath)); + } + + Set expectedKeys = recordsToRecordKeySet(expectedRecords); + assertEquals(records.size(), expectedKeys.size()); + for (GenericRecord record : records) { + String recordKey = record.get(HoodieRecord.RECORD_KEY_METADATA_FIELD).toString(); + assertEquals(commitTime, + record.get(HoodieRecord.COMMIT_TIME_METADATA_FIELD).toString()); + assertTrue(expectedKeys.contains(recordKey)); + } + } + private Pair, List> testUpdates(String instantTime, HoodieWriteClient client, int sizeToInsertAndUpdate, int expectedTotalRecords) throws IOException { diff --git a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java index 5785fc89b7702..46e117a87a07c 100644 --- a/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java +++ b/hudi-client/src/test/java/org/apache/hudi/io/TestHoodieTimelineArchiveLog.java @@ -18,7 +18,13 @@ package org.apache.hudi.io; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.fs.HoodieWrapperFileSystem; import org.apache.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.WriteOperationType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; @@ -27,14 +33,12 @@ import org.apache.hudi.common.table.timeline.HoodieInstant.State; import org.apache.hudi.common.table.timeline.HoodieTimeline; 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.config.HoodieCompactionConfig; import org.apache.hudi.config.HoodieWriteConfig; import org.apache.hudi.table.HoodieTimelineArchiveLog; import org.apache.hudi.testutils.HoodieClientTestHarness; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.Path; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -54,17 +58,17 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness { private Configuration hadoopConf; - private HoodieTableMetaClient metaClient; + private HoodieWrapperFileSystem wrapperFs; @BeforeEach public void init() throws Exception { - initDFS(); initPath(); initSparkContexts(); - hadoopConf = dfs.getConf(); - hadoopConf.addResource(dfs.getConf()); - dfs.mkdirs(new Path(basePath)); + initMetaClient(); + hadoopConf = metaClient.getHadoopConf(); + metaClient.getFs().mkdirs(new Path(basePath)); metaClient = HoodieTestUtils.init(hadoopConf, basePath); + wrapperFs = metaClient.getFs(); } @AfterEach @@ -92,58 +96,58 @@ public void testArchiveTableWithArchival() throws IOException { HoodieTestUtils.init(hadoopConf, basePath); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "100", dfs.getConf()); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "101", dfs.getConf()); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "102", dfs.getConf()); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", dfs.getConf()); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", dfs.getConf()); - HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", dfs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "100", wrapperFs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "101", wrapperFs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "102", wrapperFs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "103", wrapperFs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "104", wrapperFs.getConf()); + HoodieTestUtils.createCleanFiles(metaClient, basePath, "105", wrapperFs.getConf()); HoodieTestUtils.createPendingCleanFiles(metaClient, "106", "107"); // reload the timeline and get all the commmits before archive @@ -207,6 +211,40 @@ public void testArchiveTableWithArchival() throws IOException { verifyInflightInstants(metaClient, 2); } + @Test + public void testArchiveTableWithReplacedFiles() throws Exception { + HoodieTestUtils.init(hadoopConf, basePath); + HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) + .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") + .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 3).build()) + .build(); + + int numCommits = 4; + int commitInstant = 100; + for (int i = 0; i < numCommits; i++) { + createReplaceMetadata(commitInstant); + commitInstant += 100; + } + + metaClient = HoodieTableMetaClient.reload(metaClient); + HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); + assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match"); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); + boolean result = archiveLog.archiveIfRequired(jsc); + assertTrue(result); + + FileStatus[] allFiles = metaClient.getFs().listStatus(new Path(basePath + "/" + HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH)); + Set allFileIds = Arrays.stream(allFiles).map(fs -> FSUtils.getFileIdFromFilePath(fs.getPath())).collect(Collectors.toSet()); + + // verify 100-1,200-1 are deleted by archival + assertFalse(allFileIds.contains("file-100-1")); + assertFalse(allFileIds.contains("file-200-1")); + assertTrue(allFileIds.contains("file-100-2")); + assertTrue(allFileIds.contains("file-200-2")); + assertTrue(allFileIds.contains("file-300-1")); + assertTrue(allFileIds.contains("file-400-1")); + } + @Test public void testArchiveTableWithNoArchival() throws IOException { HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath) @@ -217,32 +255,32 @@ public void testArchiveTableWithNoArchival() throws IOException { HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); // Requested Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); // Inflight Compaction HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); + new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(4, timeline.countInstants(), "Loaded 4 commits and the count should match"); @@ -280,12 +318,12 @@ public void testArchiveCommitSafety() throws IOException { .build(); metaClient = HoodieTableMetaClient.reload(metaClient); HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); - HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match"); @@ -304,13 +342,13 @@ public void testArchiveCommitSavepointNoHole() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); - HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "101", dfs.getConf()); - HoodieTestDataGenerator.createSavepointFile(basePath, "101", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "104", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "101", wrapperFs.getConf()); + HoodieTestDataGenerator.createSavepointFile(basePath, "101", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "104", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, hadoopConf); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants(); @@ -333,19 +371,19 @@ public void testArchiveCommitCompactionNoHole() throws IOException { .withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable("test-trip-table") .withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 5).build()) .build(); - HoodieTestDataGenerator.createCommitFile(basePath, "100", dfs.getConf()); - HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf()); + HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", wrapperFs.getConf()); HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "102", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "103", dfs.getConf()); - HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "102", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "103", wrapperFs.getConf()); + HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "104", wrapperFs.getConf()); HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath, - new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "105", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "106", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "107", dfs.getConf()); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, dfs.getConf()); + new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104"), wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf()); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf()); HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsAndCompactionTimeline(); assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match"); @@ -382,27 +420,27 @@ public void testArchiveCommitTimeline() throws IOException { .build(); metaClient = HoodieTableMetaClient.reload(metaClient); - HoodieTestDataGenerator.createCommitFile(basePath, "1", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "1", wrapperFs.getConf()); HoodieInstant instant1 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "1"); - HoodieTestDataGenerator.createCommitFile(basePath, "2", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "2", wrapperFs.getConf()); Path markerPath = new Path(metaClient.getMarkerFolderPath("2")); - dfs.mkdirs(markerPath); + wrapperFs.mkdirs(markerPath); HoodieInstant instant2 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "2"); - HoodieTestDataGenerator.createCommitFile(basePath, "3", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "3", wrapperFs.getConf()); HoodieInstant instant3 = new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "3"); //add 2 more instants to pass filter criteria set in compaction config above - HoodieTestDataGenerator.createCommitFile(basePath, "4", dfs.getConf()); - HoodieTestDataGenerator.createCommitFile(basePath, "5", dfs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "4", wrapperFs.getConf()); + HoodieTestDataGenerator.createCommitFile(basePath, "5", wrapperFs.getConf()); - HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, dfs.getConf()); + HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, wrapperFs.getConf()); boolean result = archiveLog.archiveIfRequired(jsc); assertTrue(result); HoodieArchivedTimeline archivedTimeline = metaClient.getArchivedTimeline(); List archivedInstants = Arrays.asList(instant1, instant2, instant3); assertEquals(new HashSet<>(archivedInstants), archivedTimeline.getInstants().collect(Collectors.toSet())); - assertFalse(dfs.exists(markerPath)); + assertFalse(wrapperFs.exists(markerPath)); } private void verifyInflightInstants(HoodieTableMetaClient metaClient, int expectedTotalInstants) { @@ -427,4 +465,18 @@ public void testConvertCommitMetadata() { org.apache.hudi.avro.model.HoodieCommitMetadata expectedCommitMetadata = archiveLog.convertCommitMetadata(hoodieCommitMetadata); assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString()); } + + private void createReplaceMetadata(int commitInstant) throws Exception { + String commitTime = "" + commitInstant; + String fileId1 = "file-" + commitInstant + "-1"; + String fileId2 = "file-" + commitInstant + "-2"; + + // create replace instant to mark fileId1 as deleted + HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata(); + replaceMetadata.addReplaceFileId(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1); + replaceMetadata.setOperationType(WriteOperationType.INSERT_OVERWRITE); + HoodieTestTable testTable = HoodieTestTable.of(metaClient); + testTable.addReplaceCommit(commitTime, replaceMetadata); + testTable.withBaseFilesInPartition(HoodieTestDataGenerator.DEFAULT_FIRST_PARTITION_PATH, fileId1, fileId2); + } } diff --git a/hudi-common/pom.xml b/hudi-common/pom.xml index 61dbb8ea293a9..be06b8d722fce 100644 --- a/hudi-common/pom.xml +++ b/hudi-common/pom.xml @@ -73,8 +73,10 @@ ${basedir}/src/main/avro/HoodieCleanPartitionMetadata.avsc ${basedir}/src/main/avro/HoodieCleanMetadata.avsc ${basedir}/src/main/avro/HoodieCleanerPlan.avsc + ${basedir}/src/main/avro/HoodieInstantInfo.avsc ${basedir}/src/main/avro/HoodieRollbackMetadata.avsc ${basedir}/src/main/avro/HoodieRestoreMetadata.avsc + ${basedir}/src/main/avro/HoodieReplaceCommitMetadata.avsc ${basedir}/src/main/avro/HoodieArchivedMetaEntry.avsc ${basedir}/src/main/avro/HoodiePath.avsc ${basedir}/src/main/avro/HoodieFSPermission.avsc diff --git a/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc b/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc index 0572c4dc411ba..c68ef879e7551 100644 --- a/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc +++ b/hudi-common/src/main/avro/HoodieArchivedMetaEntry.avsc @@ -96,6 +96,14 @@ "name":"actionState", "type":["null","string"], "default": null + }, + { + "name":"hoodieReplaceCommitMetadata", + "type":[ + "null", + "HoodieReplaceCommitMetadata" + ], + "default": null } ] } diff --git a/hudi-common/src/main/avro/HoodieInstantInfo.avsc b/hudi-common/src/main/avro/HoodieInstantInfo.avsc new file mode 100644 index 0000000000000..e54ddde13bdc7 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieInstantInfo.avsc @@ -0,0 +1,32 @@ +/* + * 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. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieInstantInfo", + "fields":[ + { + "name":"commitTime", + "type":"string" + }, + { + "name":"action", + "type":"string" + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieReplaceCommitMetadata.avsc b/hudi-common/src/main/avro/HoodieReplaceCommitMetadata.avsc new file mode 100644 index 0000000000000..e636635c721d5 --- /dev/null +++ b/hudi-common/src/main/avro/HoodieReplaceCommitMetadata.avsc @@ -0,0 +1,64 @@ +/* + * 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. + */ +{ + "namespace":"org.apache.hudi.avro.model", + "type":"record", + "name":"HoodieReplaceCommitMetadata", + "fields":[ + { + "name":"partitionToWriteStats", + "type":["null", { + "type":"map", + "values":{ + "type":"array", + "items":"HoodieWriteStat" + } + }], + "default": null + }, + { + "name":"extraMetadata", + "type":["null", { + "type":"map", + "values":"string" + }], + "default": null + }, + { + "name":"version", + "type":["int", "null"], + "default": 1 + }, + { + "name":"operationType", + "type": ["null","string"], + "default":null + }, + { + "name":"partitionToReplaceFileIds", + "type":["null", { + "type":"map", + "values":{ + "type":"array", + "items": "string" + } + }], + "default": null + } + ] +} diff --git a/hudi-common/src/main/avro/HoodieRestoreMetadata.avsc b/hudi-common/src/main/avro/HoodieRestoreMetadata.avsc index 28e111d37e3f0..6c6f98ff1ee59 100644 --- a/hudi-common/src/main/avro/HoodieRestoreMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieRestoreMetadata.avsc @@ -34,6 +34,16 @@ "name":"version", "type":["int", "null"], "default": 1 + }, + /* overlaps with 'instantsToRollback' field. Adding this to track action type for all the instants being rolled back. */ + { + "name": "restoreInstantInfo", + "default": null, + "type": { + "type": "array", + "default": null, + "items": "HoodieInstantInfo" + } } ] } diff --git a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc index 98d4ec68f2c32..a6bd4c20ef278 100644 --- a/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc +++ b/hudi-common/src/main/avro/HoodieRollbackMetadata.avsc @@ -39,6 +39,16 @@ "name":"version", "type":["int", "null"], "default": 1 + }, + /* overlaps with 'commitsRollback' field. Adding this to track action type for all the instants being rolled back. */ + { + "name": "instantsRollback", + "default": null, + "type": { + "type": "array", + "default": null, + "items": "HoodieInstantInfo" + } } ] } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java index a596b6f39a545..6be321c9860b4 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/ActionType.java @@ -22,5 +22,6 @@ * The supported action types. */ public enum ActionType { - commit, savepoint, compaction, clean, rollback + //TODO HUDI-1281 make deltacommit part of this + commit, savepoint, compaction, clean, rollback, replacecommit } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java index d1e2cc6adb67b..3e760f6bd77d7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieCommitMetadata.java @@ -48,9 +48,9 @@ public class HoodieCommitMetadata implements Serializable { protected Map> partitionToWriteStats; protected Boolean compacted; - private Map extraMetadata; + protected Map extraMetadata; - private WriteOperationType operationType = WriteOperationType.UNKNOWN; + protected WriteOperationType operationType = WriteOperationType.UNKNOWN; // for ser/deser public HoodieCommitMetadata() { @@ -365,7 +365,9 @@ protected static ObjectMapper getObjectMapper() { @Override public String toString() { - return "HoodieCommitMetadata{partitionToWriteStats=" + partitionToWriteStats + ", compacted=" + compacted - + ", extraMetadata=" + extraMetadata + '}'; + return "HoodieCommitMetadata{" + "partitionToWriteStats=" + partitionToWriteStats + + ", compacted=" + compacted + + ", extraMetadata=" + extraMetadata + + ", operationType=" + operationType + '}'; } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java new file mode 100644 index 0000000000000..7cc9ee3a0c146 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/HoodieReplaceCommitMetadata.java @@ -0,0 +1,142 @@ +/* + * 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 com.fasterxml.jackson.annotation.JsonAutoDetect; +import com.fasterxml.jackson.annotation.JsonIgnoreProperties; +import com.fasterxml.jackson.annotation.PropertyAccessor; +import com.fasterxml.jackson.databind.DeserializationFeature; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +/** + * All the metadata that gets stored along with a commit. + */ +@JsonIgnoreProperties(ignoreUnknown = true) +public class HoodieReplaceCommitMetadata extends HoodieCommitMetadata { + private static final Logger LOG = LogManager.getLogger(HoodieReplaceCommitMetadata.class); + protected Map> partitionToReplaceFileIds; + + // for ser/deser + public HoodieReplaceCommitMetadata() { + this(false); + } + + public HoodieReplaceCommitMetadata(boolean compacted) { + super(compacted); + partitionToReplaceFileIds = new HashMap<>(); + } + + public void setPartitionToReplaceFileIds(Map> partitionToReplaceFileIds) { + this.partitionToReplaceFileIds = partitionToReplaceFileIds; + } + + public void addReplaceFileId(String partitionPath, String fileId) { + if (!partitionToReplaceFileIds.containsKey(partitionPath)) { + partitionToReplaceFileIds.put(partitionPath, new ArrayList<>()); + } + partitionToReplaceFileIds.get(partitionPath).add(fileId); + } + + public List getReplaceFileIds(String partitionPath) { + return partitionToReplaceFileIds.get(partitionPath); + } + + public Map> getPartitionToReplaceFileIds() { + return partitionToReplaceFileIds; + } + + @Override + public String toJsonString() throws IOException { + if (partitionToWriteStats.containsKey(null)) { + LOG.info("partition path is null for " + partitionToWriteStats.get(null)); + partitionToWriteStats.remove(null); + } + if (partitionToReplaceFileIds.containsKey(null)) { + LOG.info("partition path is null for " + partitionToReplaceFileIds.get(null)); + partitionToReplaceFileIds.remove(null); + } + return 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 somethings bad happen). + return clazz.newInstance(); + } + return getObjectMapper().readValue(jsonStr, clazz); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + + HoodieReplaceCommitMetadata that = (HoodieReplaceCommitMetadata) o; + + if (!partitionToWriteStats.equals(that.partitionToWriteStats)) { + return false; + } + return compacted.equals(that.compacted); + + } + + @Override + public int hashCode() { + int result = partitionToWriteStats.hashCode(); + result = 31 * result + compacted.hashCode(); + return result; + } + + public static T fromBytes(byte[] bytes, Class clazz) throws IOException { + try { + return fromJsonString(new String(bytes, StandardCharsets.UTF_8), clazz); + } catch (Exception e) { + throw new IOException("unable to read commit metadata", e); + } + } + + protected static ObjectMapper getObjectMapper() { + ObjectMapper mapper = new ObjectMapper(); + mapper.disable(DeserializationFeature.FAIL_ON_UNKNOWN_PROPERTIES); + mapper.setVisibility(PropertyAccessor.FIELD, JsonAutoDetect.Visibility.ANY); + return mapper; + } + + @Override + public String toString() { + return "HoodieReplaceMetadata{" + "partitionToWriteStats=" + partitionToWriteStats + + ", partitionToReplaceFileIds=" + partitionToReplaceFileIds + + ", compacted=" + compacted + + ", extraMetadata=" + extraMetadata + + ", operationType=" + operationType + '}'; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java index 1521c5f7a5e7b..71e0ff2abbb90 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/model/WriteOperationType.java @@ -38,6 +38,8 @@ public enum WriteOperationType { // delete DELETE("delete"), BOOTSTRAP("bootstrap"), + // insert overwrite + INSERT_OVERWRITE("insert_overwrite"), // used for old version UNKNOWN("unknown"); @@ -66,6 +68,8 @@ public static WriteOperationType fromValue(String value) { return BULK_INSERT_PREPPED; case "delete": return DELETE; + case "insert_overwrite": + return INSERT_OVERWRITE; default: throw new HoodieException("Invalid value of Type."); } 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 42fb3e9df5dc8..2e8857b2f5252 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 @@ -31,6 +31,7 @@ import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineLayout; import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.exception.HoodieException; @@ -524,14 +525,7 @@ public HoodieTimeline getCommitTimeline() { * Gets the commit action type. */ public String getCommitActionType() { - switch (this.getTableType()) { - case COPY_ON_WRITE: - return HoodieActiveTimeline.COMMIT_ACTION; - case MERGE_ON_READ: - return HoodieActiveTimeline.DELTA_COMMIT_ACTION; - default: - throw new HoodieException("Could not commit on unknown table type " + this.getTableType()); - } + return CommitUtils.getCommitActionType(this.getTableType()); } /** 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 36e2b3de1e4ed..8e5b0b664ecf4 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 @@ -65,7 +65,8 @@ public class HoodieActiveTimeline extends HoodieDefaultTimeline { COMMIT_EXTENSION, INFLIGHT_COMMIT_EXTENSION, REQUESTED_COMMIT_EXTENSION, DELTA_COMMIT_EXTENSION, INFLIGHT_DELTA_COMMIT_EXTENSION, REQUESTED_DELTA_COMMIT_EXTENSION, SAVEPOINT_EXTENSION, INFLIGHT_SAVEPOINT_EXTENSION, CLEAN_EXTENSION, REQUESTED_CLEAN_EXTENSION, INFLIGHT_CLEAN_EXTENSION, - INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION)); + INFLIGHT_COMPACTION_EXTENSION, REQUESTED_COMPACTION_EXTENSION, INFLIGHT_RESTORE_EXTENSION, RESTORE_EXTENSION, + REQUESTED_REPLACE_COMMIT_EXTENSION, INFLIGHT_REPLACE_COMMIT_EXTENSION, REPLACE_COMMIT_EXTENSION)); private static final Logger LOG = LogManager.getLogger(HoodieActiveTimeline.class); protected HoodieTableMetaClient metaClient; @@ -304,6 +305,22 @@ public HoodieInstant transitionCleanRequestedToInflight(HoodieInstant requestedI return inflight; } + /** + * Transition Clean State from inflight to Committed. + * + * @param inflightInstant Inflight instant + * @param data Extra Metadata + * @return commit instant + */ + public HoodieInstant transitionReplaceInflightToComplete(HoodieInstant inflightInstant, Option data) { + ValidationUtils.checkArgument(inflightInstant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)); + ValidationUtils.checkArgument(inflightInstant.isInflight()); + HoodieInstant commitInstant = new HoodieInstant(State.COMPLETED, REPLACE_COMMIT_ACTION, inflightInstant.getTimestamp()); + // Then write to timeline + transitionState(inflightInstant, commitInstant, data); + return commitInstant; + } + private void transitionState(HoodieInstant fromInstant, HoodieInstant toInstant, Option data) { transitionState(fromInstant, toInstant, data, false); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java index 678d056f5f5e0..8ced025af35cb 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieDefaultTimeline.java @@ -27,7 +27,6 @@ import java.io.Serializable; import java.security.MessageDigest; import java.security.NoSuchAlgorithmException; -import java.util.Collections; import java.util.List; import java.util.Set; import java.util.function.Function; @@ -109,10 +108,16 @@ public HoodieTimeline filterCompletedAndCompactionInstants() { @Override public HoodieDefaultTimeline getCommitsAndCompactionTimeline() { - Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION); + Set validActions = CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, COMPACTION_ACTION, REPLACE_COMMIT_ACTION); return new HoodieDefaultTimeline(instants.stream().filter(s -> validActions.contains(s.getAction())), details); } + @Override + public HoodieTimeline getCompletedReplaceTimeline() { + return new HoodieDefaultTimeline( + instants.stream().filter(s -> s.getAction().equals(REPLACE_COMMIT_ACTION)).filter(s -> s.isCompleted()), details); + } + @Override public HoodieTimeline filterPendingCompactionTimeline() { return new HoodieDefaultTimeline( @@ -155,7 +160,7 @@ public HoodieTimeline filter(Predicate filter) { * Get all instants (commits, delta commits) that produce new data, in the active timeline. */ public HoodieTimeline getCommitsTimeline() { - return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION)); + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, REPLACE_COMMIT_ACTION)); } /** @@ -164,14 +169,14 @@ public HoodieTimeline getCommitsTimeline() { */ public HoodieTimeline getAllCommitsTimeline() { return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, DELTA_COMMIT_ACTION, - CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION)); + CLEAN_ACTION, COMPACTION_ACTION, SAVEPOINT_ACTION, ROLLBACK_ACTION, REPLACE_COMMIT_ACTION)); } /** * Get only pure commits (inflight and completed) in the active timeline. */ public HoodieTimeline getCommitTimeline() { - return getTimelineOfActions(Collections.singleton(COMMIT_ACTION)); + return getTimelineOfActions(CollectionUtils.createSet(COMMIT_ACTION, REPLACE_COMMIT_ACTION)); } /** 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 939ba8bbe0436..65376b48e07c5 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 @@ -166,6 +166,10 @@ public String getFileName() { } else if (HoodieTimeline.RESTORE_ACTION.equals(action)) { return isInflight() ? HoodieTimeline.makeInflightRestoreFileName(timestamp) : HoodieTimeline.makeRestoreFileName(timestamp); + } else if (HoodieTimeline.REPLACE_COMMIT_ACTION.equals(action)) { + return isInflight() ? HoodieTimeline.makeInflightReplaceFileName(timestamp) + : isRequested() ? HoodieTimeline.makeRequestedReplaceFileName(timestamp) + : HoodieTimeline.makeReplaceFileName(timestamp); } throw new IllegalArgumentException("Cannot get file name for unknown action " + action); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java index b7c405ed41a76..03ba43c115993 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/HoodieTimeline.java @@ -48,6 +48,7 @@ public interface HoodieTimeline extends Serializable { String CLEAN_ACTION = "clean"; String ROLLBACK_ACTION = "rollback"; String SAVEPOINT_ACTION = "savepoint"; + String REPLACE_COMMIT_ACTION = "replacecommit"; String INFLIGHT_EXTENSION = ".inflight"; // With Async Compaction, compaction instant can be in 3 states : // (compaction-requested), (compaction-inflight), (completed) @@ -57,7 +58,7 @@ public interface HoodieTimeline extends Serializable { String[] VALID_ACTIONS_IN_TIMELINE = {COMMIT_ACTION, DELTA_COMMIT_ACTION, CLEAN_ACTION, SAVEPOINT_ACTION, RESTORE_ACTION, ROLLBACK_ACTION, - COMPACTION_ACTION}; + COMPACTION_ACTION, REPLACE_COMMIT_ACTION}; String COMMIT_EXTENSION = "." + COMMIT_ACTION; String DELTA_COMMIT_EXTENSION = "." + DELTA_COMMIT_ACTION; @@ -78,6 +79,9 @@ public interface HoodieTimeline extends Serializable { String INFLIGHT_COMPACTION_EXTENSION = StringUtils.join(".", COMPACTION_ACTION, INFLIGHT_EXTENSION); String INFLIGHT_RESTORE_EXTENSION = "." + RESTORE_ACTION + INFLIGHT_EXTENSION; String RESTORE_EXTENSION = "." + RESTORE_ACTION; + String INFLIGHT_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + INFLIGHT_EXTENSION; + String REQUESTED_REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION + REQUESTED_EXTENSION; + String REPLACE_COMMIT_EXTENSION = "." + REPLACE_COMMIT_ACTION; String INVALID_INSTANT_TS = "0"; @@ -133,6 +137,13 @@ public interface HoodieTimeline extends Serializable { */ HoodieTimeline getCommitsAndCompactionTimeline(); + /** + * Timeline to just include replace instants that have valid (commit/deltacommit) actions. + * + * @return + */ + HoodieTimeline getCompletedReplaceTimeline(); + /** * Filter this timeline to just include requested and inflight compaction instants. * @@ -360,6 +371,18 @@ static String makeInflightRestoreFileName(String instant) { return StringUtils.join(instant, HoodieTimeline.INFLIGHT_RESTORE_EXTENSION); } + static String makeReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REPLACE_COMMIT_EXTENSION); + } + + static String makeInflightReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); + } + + static String makeRequestedReplaceFileName(String instant) { + return StringUtils.join(instant, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION); + } + static String makeDeltaFileName(String instantTime) { return instantTime + HoodieTimeline.DELTA_COMMIT_EXTENSION; } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java index f95bfc3427dd9..80e7c66be40ce 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/timeline/TimelineMetadataUtils.java @@ -21,6 +21,7 @@ import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCleanerPlan; import org.apache.hudi.avro.model.HoodieCompactionPlan; +import org.apache.hudi.avro.model.HoodieInstantInfo; import org.apache.hudi.avro.model.HoodieRestoreMetadata; import org.apache.hudi.avro.model.HoodieRollbackMetadata; import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata; @@ -46,6 +47,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.stream.Collectors; public class TimelineMetadataUtils { @@ -53,14 +55,16 @@ public class TimelineMetadataUtils { public static HoodieRestoreMetadata convertRestoreMetadata(String startRestoreTime, long durationInMs, - List commits, + List instants, Map> instantToRollbackMetadata) { - return new HoodieRestoreMetadata(startRestoreTime, durationInMs, commits, - Collections.unmodifiableMap(instantToRollbackMetadata), DEFAULT_VERSION); + return new HoodieRestoreMetadata(startRestoreTime, durationInMs, + instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), + Collections.unmodifiableMap(instantToRollbackMetadata), DEFAULT_VERSION, + instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList())); } public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbackTime, Option durationInMs, - List commits, List rollbackStats) { + List instants, List rollbackStats) { Map partitionMetadataBuilder = new HashMap<>(); int totalDeleted = 0; for (HoodieRollbackStat stat : rollbackStats) { @@ -70,8 +74,10 @@ public static HoodieRollbackMetadata convertRollbackMetadata(String startRollbac totalDeleted += stat.getSuccessDeleteFiles().size(); } - return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L), totalDeleted, commits, - Collections.unmodifiableMap(partitionMetadataBuilder), DEFAULT_VERSION); + return new HoodieRollbackMetadata(startRollbackTime, durationInMs.orElseGet(() -> -1L), totalDeleted, + instants.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()), + Collections.unmodifiableMap(partitionMetadataBuilder), DEFAULT_VERSION, + instants.stream().map(instant -> new HoodieInstantInfo(instant.getTimestamp(), instant.getAction())).collect(Collectors.toList())); } public static HoodieSavepointMetadata convertSavepointMetadata(String user, String comment, diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java index 9fa44722c261d..0c6d9adbd235c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/AbstractTableFileSystemView.java @@ -18,16 +18,19 @@ package org.apache.hudi.common.table.view; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.bootstrap.index.BootstrapIndex; import org.apache.hudi.common.fs.FSUtils; +import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.BootstrapFileMapping; import org.apache.hudi.common.model.CompactionOperation; -import org.apache.hudi.common.model.BootstrapBaseFileMapping; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -37,15 +40,15 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.io.Serializable; +import java.util.AbstractMap; import java.util.ArrayList; import java.util.Arrays; +import java.util.Comparator; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -58,6 +61,7 @@ import java.util.stream.Collectors; import java.util.stream.Stream; +import static org.apache.hudi.common.table.timeline.HoodieTimeline.GREATER_THAN_OR_EQUALS; import static org.apache.hudi.common.table.timeline.HoodieTimeline.METADATA_BOOTSTRAP_INSTANT_TS; /** @@ -97,6 +101,7 @@ private String getPartitionPathFromFilePath(String fullPath) { protected void init(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline) { this.metaClient = metaClient; refreshTimeline(visibleActiveTimeline); + resetFileGroupsReplaced(visibleCommitsAndCompactionTimeline); this.bootstrapIndex = BootstrapIndex.getBootstrapIndex(metaClient); // Load Pending Compaction Operations resetPendingCompactionOperations(CompactionUtils.getAllPendingCompactionOperations(metaClient).values().stream() @@ -196,6 +201,33 @@ protected List buildFileGroups(Stream baseFileS return fileGroups; } + /** + * Get replaced instant for each file group by looking at all commit instants. + */ + private void resetFileGroupsReplaced(HoodieTimeline timeline) { + HoodieTimer hoodieTimer = new HoodieTimer(); + hoodieTimer.startTimer(); + // for each REPLACE instant, get map of (partitionPath -> deleteFileGroup) + HoodieTimeline replacedTimeline = timeline.getCompletedReplaceTimeline(); + Stream> resultStream = replacedTimeline.getInstants().flatMap(instant -> { + try { + HoodieReplaceCommitMetadata replaceMetadata = HoodieReplaceCommitMetadata.fromBytes(metaClient.getActiveTimeline().getInstantDetails(instant).get(), + HoodieReplaceCommitMetadata.class); + + // get replace instant mapping for each partition, fileId + return replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().flatMap(entry -> entry.getValue().stream().map(e -> + new AbstractMap.SimpleEntry<>(new HoodieFileGroupId(entry.getKey(), e), instant))); + } catch (IOException e) { + throw new HoodieIOException("error reading commit metadata for " + instant); + } + }); + + Map replacedFileGroups = resultStream.collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue)); + resetReplacedFileGroups(replacedFileGroups); + LOG.info("Took " + hoodieTimer.endTimer() + " ms to read " + replacedTimeline.countInstants() + " instants, " + + replacedFileGroups.size() + " replaced file groups"); + } + /** * Clears the partition Map and reset view states. */ @@ -385,6 +417,7 @@ public final Stream getLatestBaseFiles(String partitionStr) { String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchLatestBaseFiles(partitionPath) + .filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId())) .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); } finally { readLock.unlock(); @@ -408,6 +441,7 @@ public final Stream getLatestBaseFilesBeforeOrOn(String partitio String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchAllStoredFileGroups(partitionPath) + .filter(fileGroup -> !isFileGroupReplacedBeforeOrOn(fileGroup.getFileGroupId(), maxCommitTime)) .map(fileGroup -> Option.fromJavaOptional(fileGroup.getAllBaseFiles() .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.LESSER_THAN_OR_EQUALS, maxCommitTime )) @@ -425,10 +459,14 @@ public final Option getBaseFileOn(String partitionStr, String in readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() - .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, - instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null)) - .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); + if (isFileGroupReplacedBeforeOrOn(new HoodieFileGroupId(partitionPath, fileId), instantTime)) { + return Option.empty(); + } else { + return fetchHoodieFileGroup(partitionPath, fileId).map(fileGroup -> fileGroup.getAllBaseFiles() + .filter(baseFile -> HoodieTimeline.compareTimestamps(baseFile.getCommitTime(), HoodieTimeline.EQUALS, + instantTime)).filter(df -> !isBaseFileDueToPendingCompaction(df)).findFirst().orElse(null)) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); + } } finally { readLock.unlock(); } @@ -443,8 +481,12 @@ public final Option getLatestBaseFile(String partitionStr, Strin readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestBaseFile(partitionPath, fileId) - .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); + if (isFileGroupReplaced(partitionPath, fileId)) { + return Option.empty(); + } else { + return fetchLatestBaseFile(partitionPath, fileId) + .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, fileId), df)); + } } finally { readLock.unlock(); } @@ -454,7 +496,9 @@ public final Option getLatestBaseFile(String partitionStr, Strin public final Stream getLatestBaseFilesInRange(List commitsToReturn) { try { readLock.lock(); - return fetchAllStoredFileGroups().map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional( + return fetchAllStoredFileGroups() + .filter(fileGroup -> !isFileGroupReplacedBeforeAny(fileGroup.getFileGroupId(), commitsToReturn)) + .map(fileGroup -> Pair.of(fileGroup.getFileGroupId(), Option.fromJavaOptional( fileGroup.getAllBaseFiles().filter(baseFile -> commitsToReturn.contains(baseFile.getCommitTime()) && !isBaseFileDueToPendingCompaction(baseFile)).findFirst()))).filter(p -> p.getValue().isPresent()) .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); @@ -470,6 +514,7 @@ public final Stream getAllBaseFiles(String partitionStr) { String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); return fetchAllBaseFiles(partitionPath) + .filter(df -> !isFileGroupReplaced(partitionPath, df.getFileId())) .filter(df -> visibleCommitsAndCompactionTimeline.containsOrBeforeTimelineStarts(df.getCommitTime())) .filter(df -> !isBaseFileDueToPendingCompaction(df)) .map(df -> addBootstrapBaseFileIfPresent(new HoodieFileGroupId(partitionPath, df.getFileId()), df)); @@ -484,7 +529,9 @@ public final Stream getLatestFileSlices(String partitionStr) { readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchLatestFileSlices(partitionPath).map(this::filterBaseFileAfterPendingCompaction) + return fetchLatestFileSlices(partitionPath) + .filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())) + .map(this::filterBaseFileAfterPendingCompaction) .map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); @@ -500,8 +547,12 @@ public final Option getLatestFileSlice(String partitionStr, String fi readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - Option fs = fetchLatestFileSlice(partitionPath, fileId); - return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent); + if (isFileGroupReplaced(partitionPath, fileId)) { + return Option.empty(); + } else { + Option fs = fetchLatestFileSlice(partitionPath, fileId); + return fs.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent); + } } finally { readLock.unlock(); } @@ -513,17 +564,19 @@ public final Stream getLatestUnCompactedFileSlices(String partitionSt readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - return fetchAllStoredFileGroups(partitionPath).map(fileGroup -> { - FileSlice fileSlice = fileGroup.getLatestFileSlice().get(); - // if the file-group is under compaction, pick the latest before compaction instant time. - Option> compactionWithInstantPair = - getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId()); - if (compactionWithInstantPair.isPresent()) { - String compactionInstantTime = compactionWithInstantPair.get().getLeft(); - return fileGroup.getLatestFileSliceBefore(compactionInstantTime); - } - return Option.of(fileSlice); - }).map(Option::get).map(this::addBootstrapBaseFileIfPresent); + return fetchAllStoredFileGroups(partitionPath) + .filter(fg -> !isFileGroupReplaced(fg.getFileGroupId())) + .map(fileGroup -> { + FileSlice fileSlice = fileGroup.getLatestFileSlice().get(); + // if the file-group is under compaction, pick the latest before compaction instant time. + Option> compactionWithInstantPair = + getPendingCompactionOperationWithInstant(fileSlice.getFileGroupId()); + if (compactionWithInstantPair.isPresent()) { + String compactionInstantTime = compactionWithInstantPair.get().getLeft(); + return fileGroup.getLatestFileSliceBefore(compactionInstantTime); + } + return Option.of(fileSlice); + }).map(Option::get).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -536,7 +589,8 @@ public final Stream getLatestFileSlicesBeforeOrOn(String partitionStr readLock.lock(); String partitionPath = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partitionPath); - Stream fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime); + Stream fileSliceStream = fetchLatestFileSlicesBeforeOrOn(partitionPath, maxCommitTime) + .filter(slice -> !isFileGroupReplacedBeforeOrOn(slice.getFileGroupId(), maxCommitTime)); if (includeFileSlicesInPendingCompaction) { return fileSliceStream.map(this::filterBaseFileAfterPendingCompaction).map(this::addBootstrapBaseFileIfPresent); } else { @@ -554,14 +608,16 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllStoredFileGroups(partition).map(fileGroup -> { - Option fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); - // if the file-group is under construction, pick the latest before compaction instant time. - if (fileSlice.isPresent()) { - fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get())); - } - return fileSlice; - }).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); + return fetchAllStoredFileGroups(partition) + .filter(fg -> !isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxInstantTime)) + .map(fileGroup -> { + Option fileSlice = fileGroup.getLatestFileSliceBeforeOrOn(maxInstantTime); + // if the file-group is under construction, pick the latest before compaction instant time. + if (fileSlice.isPresent()) { + fileSlice = Option.of(fetchMergedFileSlice(fileGroup, fileSlice.get())); + } + return fileSlice; + }).filter(Option::isPresent).map(Option::get).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -571,7 +627,9 @@ public final Stream getLatestMergedFileSlicesBeforeOrOn(String partit public final Stream getLatestFileSliceInRange(List commitsToReturn) { try { readLock.lock(); - return fetchLatestFileSliceInRange(commitsToReturn).map(this::addBootstrapBaseFileIfPresent); + return fetchLatestFileSliceInRange(commitsToReturn) + .filter(slice -> !isFileGroupReplacedBeforeAny(slice.getFileGroupId(), commitsToReturn)) + .map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -583,7 +641,7 @@ public final Stream getAllFileSlices(String partitionStr) { readLock.lock(); String partition = formatPartitionKey(partitionStr); ensurePartitionLoadedCorrectly(partition); - return fetchAllFileSlices(partition).map(this::addBootstrapBaseFileIfPresent); + return fetchAllFileSlices(partition).filter(slice -> !isFileGroupReplaced(slice.getFileGroupId())).map(this::addBootstrapBaseFileIfPresent); } finally { readLock.unlock(); } @@ -599,6 +657,10 @@ private String formatPartitionKey(String partitionStr) { @Override public final Stream getAllFileGroups(String partitionStr) { + return getAllFileGroupsIncludingReplaced(partitionStr).filter(fg -> !isFileGroupReplaced(fg)); + } + + private Stream getAllFileGroupsIncludingReplaced(final String partitionStr) { try { readLock.lock(); // Ensure there is consistency in handling trailing slash in partition-path. Always trim it which is what is done @@ -611,6 +673,12 @@ public final Stream getAllFileGroups(String partitionStr) { } } + @Override + public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) { + return getAllFileGroupsIncludingReplaced(partitionPath).filter(fg -> isFileGroupReplacedBeforeOrOn(fg.getFileGroupId(), maxCommitTime)); + } + + // Fetch APIs to be implemented by concrete sub-classes /** @@ -727,6 +795,26 @@ protected abstract Option> getPendingCompactio */ abstract Stream fetchAllStoredFileGroups(); + /** + * Track instant time for file groups replaced. + */ + protected abstract void resetReplacedFileGroups(final Map replacedFileGroups); + + /** + * Track instant time for new file groups replaced. + */ + protected abstract void addReplacedFileGroups(final Map replacedFileGroups); + + /** + * Remove file groups that are replaced in any of the specified instants. + */ + protected abstract void removeReplacedFileIdsAtInstants(Set instants); + + /** + * Track instant time for file groups replaced. + */ + protected abstract Option getReplaceInstant(final HoodieFileGroupId fileGroupId); + /** * Check if the view is already closed. */ @@ -772,7 +860,9 @@ protected Option getLatestBaseFile(HoodieFileGroup fileGroup) { * Fetch latest base-files across all partitions. */ private Stream fetchLatestBaseFiles() { - return fetchAllStoredFileGroups().map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg))) + return fetchAllStoredFileGroups() + .filter(fg -> !isFileGroupReplaced(fg)) + .map(fg -> Pair.of(fg.getFileGroupId(), getLatestBaseFile(fg))) .filter(p -> p.getValue().isPresent()) .map(p -> addBootstrapBaseFileIfPresent(p.getKey(), p.getValue().get())); } @@ -880,6 +970,31 @@ protected Option fetchLatestFileSlice(String partitionPath, String fi .fromJavaOptional(fetchLatestFileSlices(partitionPath).filter(fs -> fs.getFileId().equals(fileId)).findFirst()); } + private boolean isFileGroupReplaced(String partitionPath, String fileId) { + return isFileGroupReplaced(new HoodieFileGroupId(partitionPath, fileId)); + } + + private boolean isFileGroupReplaced(HoodieFileGroup fileGroup) { + return isFileGroupReplaced(fileGroup.getFileGroupId()); + } + + private boolean isFileGroupReplaced(HoodieFileGroupId fileGroup) { + return getReplaceInstant(fileGroup).isPresent(); + } + + private boolean isFileGroupReplacedBeforeAny(HoodieFileGroupId fileGroupId, List instants) { + return isFileGroupReplacedBeforeOrOn(fileGroupId, instants.stream().max(Comparator.naturalOrder()).get()); + } + + private boolean isFileGroupReplacedBeforeOrOn(HoodieFileGroupId fileGroupId, String instant) { + Option hoodieInstantOption = getReplaceInstant(fileGroupId); + if (!hoodieInstantOption.isPresent()) { + return false; + } + + return HoodieTimeline.compareTimestamps(instant, GREATER_THAN_OR_EQUALS, hoodieInstantOption.get().getTimestamp()); + } + @Override public Option getLastInstant() { return getTimeline().lastInstant(); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java index 603f88fc56552..2ba4ff57a2f58 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/FileSystemViewStorageConfig.java @@ -43,10 +43,13 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { "hoodie.filesystem.view.spillable.compaction.mem.fraction"; public static final String FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION = "hoodie.filesystem.view.spillable.bootstrap.base.file.mem.fraction"; + public static final String FILESYSTEM_VIEW_REPLACED_MEM_FRACTION = + "hoodie.filesystem.view.spillable.replaced.mem.fraction"; private static final String ROCKSDB_BASE_PATH_PROP = "hoodie.filesystem.view.rocksdb.base.path"; public static final String FILESTYSTEM_REMOTE_TIMELINE_CLIENT_TIMEOUT_SECS = "hoodie.filesystem.view.remote.timeout.secs"; + public static final FileSystemViewStorageType DEFAULT_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; public static final FileSystemViewStorageType DEFAULT_SECONDARY_VIEW_STORAGE_TYPE = FileSystemViewStorageType.MEMORY; public static final String DEFAULT_ROCKSDB_BASE_PATH = "/tmp/hoodie_timeline_rocksdb"; @@ -58,6 +61,7 @@ public class FileSystemViewStorageConfig extends DefaultHoodieConfig { public static final String DEFAULT_VIEW_SPILLABLE_DIR = "/tmp/view_map/"; private static final Double DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION = 0.01; private static final Double DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE = 0.05; + private static final Double DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS = 0.01; private static final Long DEFAULT_MAX_MEMORY_FOR_VIEW = 100 * 1024 * 1024L; // 100 MB /** @@ -116,6 +120,12 @@ public long getMaxMemoryForBootstrapBaseFile() { return reservedForExternalDataFile; } + public long getMaxMemoryForReplacedFileGroups() { + long totalMemory = Long.parseLong(props.getProperty(FILESYSTEM_VIEW_SPILLABLE_MEM)); + return new Double(totalMemory * Double.parseDouble(props.getProperty(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION))) + .longValue(); + } + public String getBaseStoreDir() { return props.getProperty(FILESYSTEM_VIEW_SPILLABLE_DIR); } @@ -233,6 +243,8 @@ public FileSystemViewStorageConfig build() { FILESYSTEM_VIEW_PENDING_COMPACTION_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_PENDING_COMPACTION.toString()); setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION), FILESYSTEM_VIEW_BOOTSTRAP_BASE_FILE_FRACTION, DEFAULT_MEM_FRACTION_FOR_EXTERNAL_DATA_FILE.toString()); + setDefaultOnCondition(props, !props.containsKey(FILESYSTEM_VIEW_REPLACED_MEM_FRACTION), + FILESYSTEM_VIEW_REPLACED_MEM_FRACTION, DEFAULT_MEM_FRACTION_FOR_REPLACED_FILEGROUPS.toString()); setDefaultOnCondition(props, !props.containsKey(ROCKSDB_BASE_PATH_PROP), ROCKSDB_BASE_PATH_PROP, DEFAULT_ROCKSDB_BASE_PATH); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java index 4cbd2e3866ca1..fcf7eccc30da6 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/HoodieTableFileSystemView.java @@ -18,17 +18,17 @@ package org.apache.hudi.common.table.view; -import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hadoop.fs.FileStatus; import org.apache.hudi.common.model.BootstrapBaseFileMapping; +import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; 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 org.apache.hudi.common.util.collection.Pair; - -import org.apache.hadoop.fs.FileStatus; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -37,6 +37,7 @@ import java.util.Collection; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -64,6 +65,11 @@ public class HoodieTableFileSystemView extends IncrementalTimelineSyncFileSystem */ protected Map fgIdToBootstrapBaseFile; + /** + * Track replace time for replaced file groups. + */ + protected Map fgIdToReplaceInstants; + /** * Flag to determine if closed. */ @@ -106,6 +112,7 @@ protected void resetViewState() { this.fgIdToPendingCompaction = null; this.partitionToFileGroupsMap = null; this.fgIdToBootstrapBaseFile = null; + this.fgIdToReplaceInstants = null; } protected Map> createPartitionToFileGroups() { @@ -122,6 +129,11 @@ protected Map createFileIdToBootstr return fileGroupIdBootstrapBaseFileMap; } + protected Map createFileIdToReplaceInstantMap(final Map replacedFileGroups) { + Map replacedFileGroupsMap = new ConcurrentHashMap<>(replacedFileGroups); + return replacedFileGroupsMap; + } + /** * Create a file system view, as of the given timeline, with the provided file statuses. */ @@ -261,6 +273,26 @@ public Stream fetchAllStoredFileGroups() { return partitionToFileGroupsMap.values().stream().flatMap(Collection::stream); } + @Override + protected void resetReplacedFileGroups(final Map replacedFileGroups) { + fgIdToReplaceInstants = createFileIdToReplaceInstantMap(replacedFileGroups); + } + + @Override + protected void addReplacedFileGroups(final Map replacedFileGroups) { + fgIdToReplaceInstants.putAll(replacedFileGroups); + } + + @Override + protected void removeReplacedFileIdsAtInstants(Set instants) { + fgIdToReplaceInstants.entrySet().removeIf(entry -> instants.contains(entry.getValue().getTimestamp())); + } + + @Override + protected Option getReplaceInstant(final HoodieFileGroupId fileGroupId) { + return Option.ofNullable(fgIdToReplaceInstants.get(fileGroupId)); + } + @Override public void close() { closed = true; @@ -268,6 +300,7 @@ public void close() { partitionToFileGroupsMap = null; fgIdToPendingCompaction = null; fgIdToBootstrapBaseFile = null; + fgIdToReplaceInstants = null; } @Override diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java index ec29e93bbaf92..9f9fd1f7ce601 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/IncrementalTimelineSyncFileSystemView.java @@ -18,6 +18,8 @@ package org.apache.hudi.common.table.view; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.avro.model.HoodieCleanMetadata; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieRestoreMetadata; @@ -28,7 +30,10 @@ import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; +import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.timeline.HoodieInstant; import org.apache.hudi.common.table.timeline.HoodieTimeline; import org.apache.hudi.common.table.timeline.TimelineDiffHelper; @@ -39,15 +44,13 @@ import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieException; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import java.io.IOException; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.stream.Collectors; /** @@ -129,6 +132,8 @@ private void runIncrementalSync(HoodieTimeline timeline, TimelineDiffResult diff addPendingCompactionInstant(timeline, instant); } else if (instant.getAction().equals(HoodieTimeline.ROLLBACK_ACTION)) { addRollbackInstant(timeline, instant); + } else if (instant.getAction().equals(HoodieTimeline.REPLACE_COMMIT_ACTION)) { + addReplaceInstant(timeline, instant); } } catch (IOException ioe) { throw new HoodieException(ioe); @@ -191,7 +196,14 @@ private void addCommitInstant(HoodieTimeline timeline, HoodieInstant instant) th LOG.info("Syncing committed instant (" + instant + ")"); HoodieCommitMetadata commitMetadata = HoodieCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieCommitMetadata.class); - commitMetadata.getPartitionToWriteStats().entrySet().stream().forEach(entry -> { + updatePartitionWriteFileGroups(commitMetadata.getPartitionToWriteStats(), timeline, instant); + LOG.info("Done Syncing committed instant (" + instant + ")"); + } + + private void updatePartitionWriteFileGroups(Map> partitionToWriteStats, + HoodieTimeline timeline, + HoodieInstant instant) { + partitionToWriteStats.entrySet().stream().forEach(entry -> { String partition = entry.getKey(); if (isPartitionAvailableInStore(partition)) { LOG.info("Syncing partition (" + partition + ") of instant (" + instant + ")"); @@ -231,6 +243,13 @@ private void addRestoreInstant(HoodieTimeline timeline, HoodieInstant instant) t removeFileSlicesForPartition(timeline, instant, e.getKey(), e.getValue().stream().map(x -> x.getValue()).collect(Collectors.toList())); }); + + if (metadata.getRestoreInstantInfo() != null) { + Set rolledbackInstants = metadata.getRestoreInstantInfo().stream() + .filter(instantInfo -> HoodieTimeline.REPLACE_COMMIT_ACTION.equals(instantInfo.getAction())) + .map(instantInfo -> instantInfo.getCommitTime()).collect(Collectors.toSet()); + removeReplacedFileIdsAtInstants(rolledbackInstants); + } LOG.info("Done Syncing restore instant (" + instant + ")"); } @@ -251,6 +270,28 @@ private void addRollbackInstant(HoodieTimeline timeline, HoodieInstant instant) LOG.info("Done Syncing rollback instant (" + instant + ")"); } + /** + * Add newly found REPLACE instant. + * + * @param timeline Hoodie Timeline + * @param instant REPLACE Instant + */ + private void addReplaceInstant(HoodieTimeline timeline, HoodieInstant instant) throws IOException { + LOG.info("Syncing replace instant (" + instant + ")"); + HoodieReplaceCommitMetadata replaceMetadata = + HoodieReplaceCommitMetadata.fromBytes(timeline.getInstantDetails(instant).get(), HoodieReplaceCommitMetadata.class); + updatePartitionWriteFileGroups(replaceMetadata.getPartitionToWriteStats(), timeline, instant); + replaceMetadata.getPartitionToReplaceFileIds().entrySet().stream().forEach(entry -> { + String partition = entry.getKey(); + Map replacedFileIds = entry.getValue().stream() + .collect(Collectors.toMap(replaceStat -> new HoodieFileGroupId(partition, replaceStat), replaceStat -> instant)); + + LOG.info("For partition (" + partition + ") of instant (" + instant + "), excluding " + replacedFileIds.size() + " file groups"); + addReplacedFileGroups(replacedFileIds); + }); + LOG.info("Done Syncing REPLACE instant (" + instant + ")"); + } + /** * Add newly found clean instant. Note that cleaner metadata (.clean.completed) * contains only relative paths unlike clean plans (.clean.requested) which contains absolute paths. diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java index 200d4216914cf..5d34b385e5c96 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/PriorityBasedFileSystemView.java @@ -193,6 +193,11 @@ public Stream getAllFileGroups(String partitionPath) { return execute(partitionPath, preferredView::getAllFileGroups, secondaryView::getAllFileGroups); } + @Override + public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) { + return execute(maxCommitTime, partitionPath, preferredView::getReplacedFileGroupsBeforeOrOn, secondaryView::getReplacedFileGroupsBeforeOrOn); + } + @Override public Stream> getPendingCompactionOperations() { return execute(preferredView::getPendingCompactionOperations, secondaryView::getPendingCompactionOperations); diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java index a0fe60c2260a2..ec62361d8c916 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RemoteHoodieTableFileSystemView.java @@ -18,6 +18,11 @@ package org.apache.hudi.common.table.view; +import com.fasterxml.jackson.core.type.TypeReference; +import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.http.client.fluent.Request; +import org.apache.http.client.fluent.Response; +import org.apache.http.client.utils.URIBuilder; import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; @@ -36,12 +41,6 @@ import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieRemoteException; - -import com.fasterxml.jackson.core.type.TypeReference; -import com.fasterxml.jackson.databind.ObjectMapper; -import org.apache.http.client.fluent.Request; -import org.apache.http.client.fluent.Response; -import org.apache.http.client.utils.URIBuilder; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -87,6 +86,9 @@ public class RemoteHoodieTableFileSystemView implements SyncableFileSystemView, public static final String ALL_FILEGROUPS_FOR_PARTITION_URL = String.format("%s/%s", BASE_URL, "filegroups/all/partition/"); + public static final String ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON = + String.format("%s/%s", BASE_URL, "filegroups/replaced/beforeoron/"); + public static final String LAST_INSTANT = String.format("%s/%s", BASE_URL, "timeline/instant/last"); public static final String LAST_INSTANTS = String.format("%s/%s", BASE_URL, "timeline/instants/last"); @@ -361,6 +363,18 @@ public Stream getAllFileGroups(String partitionPath) { } } + @Override + public Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath) { + Map paramsMap = getParamsWithAdditionalParam(partitionPath, MAX_INSTANT_PARAM, maxCommitTime); + try { + List fileGroups = executeRequest(ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON, paramsMap, + new TypeReference>() {}, RequestMethod.GET); + return fileGroups.stream().map(dto -> FileGroupDTO.toFileGroup(dto, metaClient)); + } catch (IOException e) { + throw new HoodieRemoteException(e); + } + } + public boolean refresh() { Map paramsMap = getParams(); try { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java index 29309a57d11ed..0210ae1fc7df1 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/RocksDbBasedFileSystemView.java @@ -18,23 +18,23 @@ package org.apache.hudi.common.table.view; -import org.apache.hudi.common.model.CompactionOperation; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; import org.apache.hudi.common.model.BootstrapBaseFileMapping; +import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; 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.RocksDBSchemaHelper; import org.apache.hudi.common.util.ValidationUtils; import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.common.util.collection.RocksDBDAO; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; @@ -43,6 +43,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -371,6 +372,59 @@ Option fetchHoodieFileGroup(String partitionPath, String fileId schemaHelper.getPrefixForSliceViewByPartitionFile(partitionPath, fileId)).map(Pair::getValue)).findFirst()); } + @Override + protected void resetReplacedFileGroups(final Map replacedFileGroups) { + LOG.info("Resetting replacedFileGroups to ROCKSDB based file-system view at " + + config.getRocksdbBasePath() + ", Total file-groups=" + replacedFileGroups.size()); + + // Delete all replaced file groups + rocksDB.prefixDelete(schemaHelper.getColFamilyForReplacedFileGroups(), "part="); + // Now add new entries + addReplacedFileGroups(replacedFileGroups); + LOG.info("Resetting replacedFileGroups to ROCKSDB based file-system view complete"); + } + + @Override + protected void addReplacedFileGroups(final Map replacedFileGroups) { + Map>> partitionToReplacedFileGroups = + replacedFileGroups.entrySet().stream().collect(Collectors.groupingBy(e -> e.getKey().getPartitionPath())); + partitionToReplacedFileGroups.entrySet().stream().forEach(partitionToReplacedFileGroupsEntry -> { + String partitionPath = partitionToReplacedFileGroupsEntry.getKey(); + List> replacedFileGroupsInPartition = partitionToReplacedFileGroupsEntry.getValue(); + + // Now add them + rocksDB.writeBatch(batch -> + replacedFileGroupsInPartition.stream().forEach(fgToReplacedInstant -> { + rocksDB.putInBatch(batch, schemaHelper.getColFamilyForReplacedFileGroups(), + schemaHelper.getKeyForReplacedFileGroup(fgToReplacedInstant.getKey()), fgToReplacedInstant.getValue()); + }) + ); + + LOG.info("Finished adding replaced file groups to partition (" + partitionPath + ") to ROCKSDB based view at " + + config.getRocksdbBasePath() + ", Total file-groups=" + partitionToReplacedFileGroupsEntry.getValue().size()); + }); + } + + @Override + protected void removeReplacedFileIdsAtInstants(Set instants) { + //TODO can we make this more efficient by storing reverse mapping (Instant -> FileGroupId) as well? + Stream keysToDelete = rocksDB.prefixSearch(schemaHelper.getColFamilyForReplacedFileGroups(), "") + .filter(entry -> instants.contains(entry.getValue().getTimestamp())) + .map(Pair::getKey); + + rocksDB.writeBatch(batch -> + keysToDelete.forEach(key -> rocksDB.deleteInBatch(batch, schemaHelper.getColFamilyForReplacedFileGroups(), key)) + ); + } + + @Override + protected Option getReplaceInstant(final HoodieFileGroupId fileGroupId) { + String lookupKey = schemaHelper.getKeyForReplacedFileGroup(fileGroupId); + HoodieInstant replacedInstant = + rocksDB.get(schemaHelper.getColFamilyForReplacedFileGroups(), lookupKey); + return Option.ofNullable(replacedInstant); + } + private Stream getFileGroups(Stream sliceStream) { return sliceStream.map(s -> Pair.of(Pair.of(s.getPartitionPath(), s.getFileId()), s)) .collect(Collectors.groupingBy(Pair::getKey)).entrySet().stream().map(slicePair -> { diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java index 2e136ea5ca146..9f807da824364 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/SpillableMapBasedFileSystemView.java @@ -18,24 +18,28 @@ package org.apache.hudi.common.table.view; -import java.io.File; -import java.io.IOException; -import java.util.List; -import java.util.Map; -import java.util.stream.Stream; import org.apache.hadoop.fs.FileStatus; -import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.BootstrapBaseFileMapping; +import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; 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.DefaultSizeEstimator; +import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.ExternalSpillableMap; import org.apache.hudi.common.util.collection.Pair; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; +import java.io.File; +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.stream.Stream; + /** * Table FileSystemView implementation where view is stored in spillable disk using fixed memory. */ @@ -46,6 +50,7 @@ public class SpillableMapBasedFileSystemView extends HoodieTableFileSystemView { private final long maxMemoryForFileGroupMap; private final long maxMemoryForPendingCompaction; private final long maxMemoryForBootstrapBaseFile; + private final long maxMemoryForReplaceFileGroups; private final String baseStoreDir; public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieTimeline visibleActiveTimeline, @@ -54,6 +59,7 @@ public SpillableMapBasedFileSystemView(HoodieTableMetaClient metaClient, HoodieT this.maxMemoryForFileGroupMap = config.getMaxMemoryForFileGroupMap(); this.maxMemoryForPendingCompaction = config.getMaxMemoryForPendingCompaction(); this.maxMemoryForBootstrapBaseFile = config.getMaxMemoryForBootstrapBaseFile(); + this.maxMemoryForReplaceFileGroups = config.getMaxMemoryForReplacedFileGroups(); this.baseStoreDir = config.getBaseStoreDir(); init(metaClient, visibleActiveTimeline); } @@ -109,6 +115,21 @@ protected Map createFileIdToBootstr } } + @Override + protected Map createFileIdToReplaceInstantMap(final Map replacedFileGroups) { + try { + LOG.info("Creating file group id to replace instant map using external spillable Map. Max Mem=" + maxMemoryForReplaceFileGroups + + ", BaseDir=" + baseStoreDir); + new File(baseStoreDir).mkdirs(); + Map pendingMap = new ExternalSpillableMap<>( + maxMemoryForReplaceFileGroups, baseStoreDir, new DefaultSizeEstimator(), new DefaultSizeEstimator<>()); + pendingMap.putAll(replacedFileGroups); + return pendingMap; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + @Override public Stream getAllFileGroups() { return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream() @@ -129,4 +150,18 @@ Stream fetchBootstrapBaseFiles() { public Stream fetchAllStoredFileGroups() { return ((ExternalSpillableMap) partitionToFileGroupsMap).valueStream().flatMap(fg -> ((List) fg).stream()); } + + @Override + protected void removeReplacedFileIdsAtInstants(Set instants) { + //TODO should we make this more efficient by having reverse mapping of instant to file group id? + Stream fileIdsToRemove = fgIdToReplaceInstants.entrySet().stream().map(entry -> { + if (instants.contains(entry.getValue().getTimestamp())) { + return Option.of(entry.getKey()); + } else { + return Option.ofNullable((HoodieFileGroupId) null); + } + }).filter(Option::isPresent).map(Option::get); + + fileIdsToRemove.forEach(fileGroupId -> fgIdToReplaceInstants.remove(fileGroupId)); + } } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java index 414f8e906e266..e8fe68ad4df6c 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/table/view/TableFileSystemView.java @@ -164,4 +164,9 @@ interface SliceView extends SliceViewWithLatestSlice { * Timeline corresponding to the view. */ HoodieTimeline getTimeline(); + + /** + * Stream all the replaced file groups before maxCommitTime. + */ + Stream getReplacedFileGroupsBeforeOrOn(String maxCommitTime, String partitionPath); } diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java new file mode 100644 index 0000000000000..7b4c7c5cad1f2 --- /dev/null +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/CommitUtils.java @@ -0,0 +1,96 @@ +/* + * 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.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieTableType; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieActiveTimeline; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.exception.HoodieException; +import org.apache.log4j.LogManager; +import org.apache.log4j.Logger; + +import java.util.List; +import java.util.Map; + +/** + * Helper class to generate commit metadata. + */ +public class CommitUtils { + + private static final Logger LOG = LogManager.getLogger(CommitUtils.class); + + /** + * Gets the commit action type for given table type. + */ + public static String getCommitActionType(HoodieTableType tableType) { + switch (tableType) { + case COPY_ON_WRITE: + return HoodieActiveTimeline.COMMIT_ACTION; + case MERGE_ON_READ: + return HoodieActiveTimeline.DELTA_COMMIT_ACTION; + default: + throw new HoodieException("Could not commit on unknown table type " + tableType); + } + } + + public static HoodieCommitMetadata buildMetadata(List writeStats, + Map> partitionToReplaceFileIds, + Option> extraMetadata, + WriteOperationType operationType, + String schemaToStoreInCommit, + String commitActionType) { + + HoodieCommitMetadata commitMetadata = buildMetadataFromStats(writeStats, partitionToReplaceFileIds, commitActionType, operationType); + + // add in extra metadata + if (extraMetadata.isPresent()) { + extraMetadata.get().forEach(commitMetadata::addMetadata); + } + commitMetadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, schemaToStoreInCommit); + commitMetadata.setOperationType(operationType); + return commitMetadata; + } + + private static HoodieCommitMetadata buildMetadataFromStats(List writeStats, + Map> partitionToReplaceFileIds, + String commitActionType, + WriteOperationType operationType) { + final HoodieCommitMetadata commitMetadata; + if (commitActionType == HoodieTimeline.REPLACE_COMMIT_ACTION) { + HoodieReplaceCommitMetadata replaceMetadata = new HoodieReplaceCommitMetadata(); + replaceMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + commitMetadata = replaceMetadata; + } else { + commitMetadata = new HoodieCommitMetadata(); + } + + for (HoodieWriteStat writeStat : writeStats) { + String partition = writeStat.getPartitionPath(); + commitMetadata.addWriteStat(partition, writeStat); + } + + LOG.info("Creating metadata for " + operationType + " numWriteStats:" + writeStats.size() + + "numReplaceFileIds:" + partitionToReplaceFileIds.values().stream().mapToInt(e -> e.size()).sum()); + return commitMetadata; + } +} diff --git a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java index e56dba41adfb2..be8ccefc2f5b7 100644 --- a/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java +++ b/hudi-common/src/main/java/org/apache/hudi/common/util/RocksDBSchemaHelper.java @@ -47,17 +47,19 @@ public class RocksDBSchemaHelper { private final String colFamilyForPendingCompaction; private final String colFamilyForBootstrapBaseFile; private final String colFamilyForStoredPartitions; + private final String colFamilyForReplacedFileGroups; public RocksDBSchemaHelper(HoodieTableMetaClient metaClient) { this.colFamilyForBootstrapBaseFile = "hudi_bootstrap_basefile_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForPendingCompaction = "hudi_pending_compaction_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForStoredPartitions = "hudi_partitions_" + metaClient.getBasePath().replace("/", "_"); this.colFamilyForView = "hudi_view_" + metaClient.getBasePath().replace("/", "_"); + this.colFamilyForReplacedFileGroups = "hudi_replaced_fg" + metaClient.getBasePath().replace("/", "_"); } public List getAllColumnFamilies() { return Arrays.asList(getColFamilyForView(), getColFamilyForPendingCompaction(), getColFamilyForBootstrapBaseFile(), - getColFamilyForStoredPartitions()); + getColFamilyForStoredPartitions(), getColFamilyForReplacedFileGroups()); } public String getKeyForPartitionLookup(String partition) { @@ -72,6 +74,10 @@ public String getKeyForBootstrapBaseFile(HoodieFileGroupId fgId) { return getPartitionFileIdBasedLookup(fgId); } + public String getKeyForReplacedFileGroup(HoodieFileGroupId fgId) { + return getPartitionFileIdBasedLookup(fgId); + } + public String getKeyForSliceView(HoodieFileGroup fileGroup, FileSlice slice) { return getKeyForSliceView(fileGroup.getPartitionPath(), fileGroup.getFileGroupId().getFileId(), slice.getBaseInstantTime()); @@ -125,4 +131,8 @@ public String getColFamilyForBootstrapBaseFile() { public String getColFamilyForStoredPartitions() { return colFamilyForStoredPartitions; } + + public String getColFamilyForReplacedFileGroups() { + return colFamilyForReplacedFileGroups; + } } diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java index 2ea418c480b87..e677f491f14d5 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/TestTimelineUtils.java @@ -135,7 +135,7 @@ public void testRestoreInstants() throws Exception { String ts = i + ""; HoodieInstant instant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, ts); activeTimeline.createNewInstant(instant); - activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2))); + activeTimeline.saveAsComplete(instant, Option.of(getRestoreMetadata(basePath, ts, ts, 2, HoodieTimeline.COMMIT_ACTION))); } metaClient.reloadActiveTimeline(); @@ -184,25 +184,23 @@ public void testGetExtraMetadata() throws Exception { assertEquals(extraMetadataValue1, extraMetadataEntries.get("1").get()); } - private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count) throws IOException { - HoodieRestoreMetadata metadata = new HoodieRestoreMetadata(); + private byte[] getRestoreMetadata(String basePath, String partition, String commitTs, int count, String actionType) throws IOException { List rollbackM = new ArrayList<>(); - rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count)); - metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(commitTs, rollbackM)); - List rollbackInstants = new ArrayList<>(); - rollbackInstants.add(commitTs); - metadata.setInstantsToRollback(rollbackInstants); - metadata.setStartRestoreTime(commitTs); + rollbackM.add(getRollbackMetadataInstance(basePath, partition, commitTs, count, actionType)); + List rollbackInstants = new ArrayList<>(); + rollbackInstants.add(new HoodieInstant(false, commitTs, actionType)); + HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(commitTs, 200, rollbackInstants, + CollectionUtils.createImmutableMap(commitTs, rollbackM)); return TimelineMetadataUtils.serializeRestoreMetadata(metadata).get(); } - private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count) { + private HoodieRollbackMetadata getRollbackMetadataInstance(String basePath, String partition, String commitTs, int count, String actionType) { List deletedFiles = new ArrayList<>(); for (int i = 1; i <= count; i++) { deletedFiles.add("file-" + i); } - List rollbacks = new ArrayList<>(); - rollbacks.add(commitTs); + List rollbacks = new ArrayList<>(); + rollbacks.add(new HoodieInstant(false, actionType, commitTs)); HoodieRollbackStat rollbackStat = new HoodieRollbackStat(partition, deletedFiles, Collections.emptyList(), Collections.emptyMap()); List rollbackStats = new ArrayList<>(); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java index 8a23491d4a122..d7e3bde8cb074 100755 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/timeline/TestHoodieActiveTimeline.java @@ -199,10 +199,10 @@ public void testTimelineGetOperations() { // Test that various types of getXXX operations from HoodieActiveTimeline // return the correct set of Instant checkTimeline.accept(timeline.getCommitsTimeline(), - CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION)); + CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getCommitsAndCompactionTimeline(), - CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION)); - checkTimeline.accept(timeline.getCommitTimeline(), Collections.singleton(HoodieTimeline.COMMIT_ACTION)); + CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); + checkTimeline.accept(timeline.getCommitTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION)); checkTimeline.accept(timeline.getDeltaCommitTimeline(), Collections.singleton(HoodieTimeline.DELTA_COMMIT_ACTION)); checkTimeline.accept(timeline.getCleanerTimeline(), Collections.singleton(HoodieTimeline.CLEAN_ACTION)); checkTimeline.accept(timeline.getRollbackTimeline(), Collections.singleton(HoodieTimeline.ROLLBACK_ACTION)); @@ -210,7 +210,7 @@ public void testTimelineGetOperations() { checkTimeline.accept(timeline.getSavePointTimeline(), Collections.singleton(HoodieTimeline.SAVEPOINT_ACTION)); checkTimeline.accept(timeline.getAllCommitsTimeline(), CollectionUtils.createSet(HoodieTimeline.COMMIT_ACTION, HoodieTimeline.DELTA_COMMIT_ACTION, - HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, + HoodieTimeline.CLEAN_ACTION, HoodieTimeline.COMPACTION_ACTION, HoodieTimeline.REPLACE_COMMIT_ACTION, HoodieTimeline.SAVEPOINT_ACTION, HoodieTimeline.ROLLBACK_ACTION)); // Get some random Instants @@ -404,6 +404,27 @@ public void testFiltering() { .forEach(i -> assertFalse(t2.containsInstant(i))); } + @Test + public void testReplaceActionsTimeline() { + int instantTime = 1; + List allInstants = new ArrayList<>(); + HoodieInstant instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); + allInstants.add(instant); + instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, String.format("%03d", instantTime++)); + allInstants.add(instant); + instant = new HoodieInstant(State.COMPLETED, HoodieTimeline.REPLACE_COMMIT_ACTION, String.format("%03d", instantTime++)); + allInstants.add(instant); + + timeline = new HoodieActiveTimeline(metaClient); + timeline.setInstants(allInstants); + List validReplaceInstants = + timeline.getCompletedReplaceTimeline().getInstants().collect(Collectors.toList()); + + assertEquals(1, validReplaceInstants.size()); + assertEquals(instant.getTimestamp(), validReplaceInstants.get(0).getTimestamp()); + assertEquals(HoodieTimeline.REPLACE_COMMIT_ACTION, validReplaceInstants.get(0).getAction()); + } + /** * Returns an exhaustive list of all possible HoodieInstant. * @return list of HoodieInstant diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java index 3914c813f00eb..c58df38c0a8e9 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestHoodieTableFileSystemView.java @@ -18,6 +18,9 @@ package org.apache.hudi.common.table.view; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsAction; import org.apache.hudi.avro.model.HoodieCompactionPlan; import org.apache.hudi.avro.model.HoodieFSPermission; import org.apache.hudi.avro.model.HoodieFileStatus; @@ -31,10 +34,12 @@ import org.apache.hudi.common.model.CompactionOperation; import org.apache.hudi.common.model.FileSlice; import org.apache.hudi.common.model.HoodieBaseFile; +import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; import org.apache.hudi.common.model.HoodieLogFile; import org.apache.hudi.common.model.HoodieTableType; +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; @@ -44,13 +49,10 @@ import org.apache.hudi.common.table.view.TableFileSystemView.SliceView; import org.apache.hudi.common.testutils.HoodieCommonTestHarness; import org.apache.hudi.common.testutils.HoodieTestUtils; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.CompactionUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.collection.Pair; - -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.permission.FsAction; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.BeforeEach; @@ -61,11 +63,13 @@ import java.io.File; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Paths; import java.util.ArrayList; import java.util.Arrays; - +import java.util.Collections; import java.util.Date; +import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -1273,6 +1277,153 @@ private static void saveAsComplete(HoodieActiveTimeline timeline, HoodieInstant } } + @Test + public void testReplaceWithTimeTravel() throws IOException { + String partitionPath1 = "2020/06/27"; + new File(basePath + "/" + partitionPath1).mkdirs(); + + // create 2 fileId in partition1 - fileId1 is replaced later on. + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + + assertFalse(roView.getLatestBaseFiles(partitionPath1) + .anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)), + "No commit, should not find any data file"); + // Only one commit + String commitTime1 = "1"; + String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, commitTime1); + saveAsComplete(commitTimeline, instant1, Option.empty()); + refreshFsView(); + assertEquals(1, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); + assertEquals(1, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId2)).count()); + + // create commit2 - fileId1 is replaced. new file groups fileId3,fileId4 are created. + String fileId3 = UUID.randomUUID().toString(); + String fileId4 = UUID.randomUUID().toString(); + String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + new File(basePath + "/" + partitionPath1 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName4).createNewFile(); + + String commitTime2 = "2"; + Map> partitionToReplaceFileIds = new HashMap<>(); + List replacedFileIds = new ArrayList<>(); + replacedFileIds.add(fileId1); + partitionToReplaceFileIds.put(partitionPath1, replacedFileIds); + HoodieCommitMetadata commitMetadata = + CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); + commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant2 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime2); + saveAsComplete(commitTimeline, instant2, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + //make sure view doesnt include fileId1 + refreshFsView(); + assertEquals(0, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); + assertEquals(1, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId2)).count()); + assertEquals(1, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId3)).count()); + assertEquals(1, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId4)).count()); + + //exclude commit 2 and make sure fileId1 shows up in view. + SyncableFileSystemView filteredView = getFileSystemView(metaClient.getActiveTimeline().findInstantsBefore("2"), false); + assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); + assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId2)).count()); + assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId3)).count()); + assertEquals(1, filteredView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId4)).count()); + + // ensure replacedFileGroupsBefore works with all instants + List replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("1", partitionPath1).collect(Collectors.toList()); + assertEquals(0, replacedOnInstant1.size()); + + List allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList()); + assertEquals(1, allReplaced.size()); + assertEquals(fileId1, allReplaced.get(0).getFileGroupId().getFileId()); + } + + @Test + public void testReplaceFileIdIsExcludedInView() throws IOException { + String partitionPath1 = "2020/06/27"; + String partitionPath2 = "2020/07/14"; + new File(basePath + "/" + partitionPath1).mkdirs(); + new File(basePath + "/" + partitionPath2).mkdirs(); + + // create 2 fileId in partition1 - fileId1 is replaced later on. + String fileId1 = UUID.randomUUID().toString(); + String fileId2 = UUID.randomUUID().toString(); + + // create 2 fileId in partition2 - fileId3, fileId4 is replaced later on. + String fileId3 = UUID.randomUUID().toString(); + String fileId4 = UUID.randomUUID().toString(); + + assertFalse(roView.getLatestBaseFiles(partitionPath1) + .anyMatch(dfile -> dfile.getFileId().equals(fileId1) || dfile.getFileId().equals(fileId2)), + "No commit, should not find any data file"); + assertFalse(roView.getLatestBaseFiles(partitionPath2) + .anyMatch(dfile -> dfile.getFileId().equals(fileId3) || dfile.getFileId().equals(fileId4)), + "No commit, should not find any data file"); + + // Only one commit + String commitTime1 = "1"; + String fileName1 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId1); + String fileName2 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId2); + String fileName3 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId3); + String fileName4 = FSUtils.makeDataFileName(commitTime1, TEST_WRITE_TOKEN, fileId4); + new File(basePath + "/" + partitionPath1 + "/" + fileName1).createNewFile(); + new File(basePath + "/" + partitionPath1 + "/" + fileName2).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName3).createNewFile(); + new File(basePath + "/" + partitionPath2 + "/" + fileName4).createNewFile(); + + Map> partitionToReplaceFileIds = new HashMap<>(); + List replacedFileIdsP1 = new ArrayList<>(); + replacedFileIdsP1.add(fileId1); + partitionToReplaceFileIds.put(partitionPath1, replacedFileIdsP1); + List replacedFileIdsP2 = new ArrayList<>(); + replacedFileIdsP2.add(fileId3); + replacedFileIdsP2.add(fileId4); + partitionToReplaceFileIds.put(partitionPath2, replacedFileIdsP2); + HoodieCommitMetadata commitMetadata = + CommitUtils.buildMetadata(Collections.emptyList(), partitionToReplaceFileIds, Option.empty(), WriteOperationType.INSERT_OVERWRITE, "", HoodieTimeline.REPLACE_COMMIT_ACTION); + + HoodieActiveTimeline commitTimeline = metaClient.getActiveTimeline(); + HoodieInstant instant1 = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, commitTime1); + saveAsComplete(commitTimeline, instant1, Option.of(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + refreshFsView(); + assertEquals(0, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId1)).count()); + assertEquals(fileName2, roView.getLatestBaseFiles(partitionPath1) + .filter(dfile -> dfile.getFileId().equals(fileId2)).findFirst().get().getFileName()); + assertEquals(0, roView.getLatestBaseFiles(partitionPath2) + .filter(dfile -> dfile.getFileId().equals(fileId3)).count()); + assertEquals(0, roView.getLatestBaseFiles(partitionPath2) + .filter(dfile -> dfile.getFileId().equals(fileId4)).count()); + + // ensure replacedFileGroupsBefore works with all instants + List replacedOnInstant1 = fsView.getReplacedFileGroupsBeforeOrOn("0", partitionPath1).collect(Collectors.toList()); + assertEquals(0, replacedOnInstant1.size()); + + List allReplaced = fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath1).collect(Collectors.toList()); + allReplaced.addAll(fsView.getReplacedFileGroupsBeforeOrOn("2", partitionPath2).collect(Collectors.toList())); + assertEquals(3, allReplaced.size()); + Set allReplacedFileIds = allReplaced.stream().map(fg -> fg.getFileGroupId().getFileId()).collect(Collectors.toSet()); + Set actualReplacedFileIds = Stream.of(fileId1, fileId3, fileId4).collect(Collectors.toSet()); + assertEquals(actualReplacedFileIds, allReplacedFileIds); + } + @Override protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java index 9ffa7f8c5eaa4..e4933cf4e983b 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/table/view/TestIncrementalFSViewSync.java @@ -32,6 +32,7 @@ import org.apache.hudi.common.model.HoodieCommitMetadata; import org.apache.hudi.common.model.HoodieFileGroup; import org.apache.hudi.common.model.HoodieFileGroupId; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.table.HoodieTableMetaClient; @@ -49,6 +50,7 @@ import org.apache.hudi.exception.HoodieException; import org.apache.hadoop.fs.Path; +import org.apache.hudi.exception.HoodieIOException; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.junit.jupiter.api.BeforeEach; @@ -62,6 +64,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; +import java.util.Comparator; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -81,12 +84,13 @@ public class TestIncrementalFSViewSync extends HoodieCommonTestHarness { private static final Logger LOG = LogManager.getLogger(TestIncrementalFSViewSync.class); + private static final int NUM_FILE_IDS_PER_PARTITION = 10; private static String TEST_WRITE_TOKEN = "1-0-1"; private final List partitions = Arrays.asList("2018/01/01", "2018/01/02", "2019/03/01"); private final List fileIdsPerPartition = - IntStream.range(0, 10).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList()); + IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList()); @BeforeEach public void init() throws IOException { @@ -143,13 +147,13 @@ public void testAsyncCompaction() throws IOException { { put("11", Arrays.asList("12", "13", "15")); } - }, instantsToFiles, Collections.singletonList("11")); + }, instantsToFiles, Collections.singletonList("11"), 0, 0); // Add one more ingestion instant. This should be 2nd slice now instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("22"), true, "19", 2)); // Restore last ingestion - testRestore(view, Collections.singletonList("23"), true, new HashMap<>(), Collections.singletonList("22"), "24", false); + testRestore(view, Collections.singletonList("23"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("22", true)), "24", false); // Run one more ingestion. THis is still 2nd slice instantsToFiles.putAll(testMultipleWriteSteps(view, Collections.singletonList("24"), true, "19", 2)); @@ -188,14 +192,117 @@ public void testIngestion() throws IOException { Map> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12", "13", "14")); // restore instants in reverse order till we rollback all - testRestore(view, Arrays.asList("15", "16", "17"), false, instantsToFiles, Arrays.asList("14", "13", "12"), "17", - true); + testRestore(view, Arrays.asList("15", "16", "17"), instantsToFiles, + Arrays.asList(getHoodieCommitInstant("14", false), getHoodieCommitInstant("13", false), getHoodieCommitInstant("12", false)), + "17", true); // Add 5 non-empty ingestions back-to-back instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("18", "19", "20")); // Clean instants. - testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19")); + testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), 0, 0); + } + + @Test + public void testReplaceCommits() throws IOException { + SyncableFileSystemView view = getFileSystemView(metaClient); + + // Add an empty ingestion + String firstEmptyInstantTs = "11"; + HoodieCommitMetadata metadata = new HoodieCommitMetadata(); + metaClient.getActiveTimeline().createNewInstant( + new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs)); + metaClient.getActiveTimeline().saveAsComplete( + new HoodieInstant(true, HoodieTimeline.COMMIT_ACTION, firstEmptyInstantTs), + Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + + view.sync(); + assertTrue(view.getLastInstant().isPresent()); + assertEquals("11", view.getLastInstant().get().getTimestamp()); + assertEquals(State.COMPLETED, view.getLastInstant().get().getState()); + assertEquals(HoodieTimeline.COMMIT_ACTION, view.getLastInstant().get().getAction()); + partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count())); + + metaClient.reloadActiveTimeline(); + SyncableFileSystemView newView = getFileSystemView(metaClient); + + areViewsConsistent(view, newView, 0L); + + // Add 1 non-empty ingestions to COW table + Map> instantsToFiles = testMultipleWriteSteps(view, Arrays.asList("12")); + + // ADD replace instants + testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("13", "14"), NUM_FILE_IDS_PER_PARTITION); + + // restore instants in reverse order till we rollback all replace instants + testRestore(view, Arrays.asList("15", "16"), instantsToFiles, + Arrays.asList(getHoodieReplaceInstant("14"), getHoodieReplaceInstant("13")), + "17", true, 1, fileIdsPerPartition.size()); + + // clear files from inmemory view for replaced instants + instantsToFiles.remove("14"); + instantsToFiles.remove("13"); + + // add few more replace instants + testMultipleReplaceSteps(instantsToFiles, view, Arrays.asList("18", "19", "20"), NUM_FILE_IDS_PER_PARTITION); + + // Clean instants. + testCleans(view, Arrays.asList("21", "22"), instantsToFiles, Arrays.asList("18", "19"), NUM_FILE_IDS_PER_PARTITION, 1); + } + + private void testMultipleReplaceSteps(Map> instantsToFiles, SyncableFileSystemView view, List instants, + int initialExpectedSlicesPerPartition) { + int expectedSlicesPerPartition = initialExpectedSlicesPerPartition; + for (int i = 0; i < instants.size(); i++) { + try { + generateReplaceInstant(instants.get(i), instantsToFiles); + view.sync(); + + metaClient.reloadActiveTimeline(); + SyncableFileSystemView newView = getFileSystemView(metaClient); + // 1 fileId is replaced for every partition, so subtract partitions.size() + expectedSlicesPerPartition = expectedSlicesPerPartition + fileIdsPerPartition.size() - 1; + areViewsConsistent(view, newView, expectedSlicesPerPartition * partitions.size()); + } catch (IOException e) { + throw new HoodieIOException("unable to test replace", e); + } + } + } + + private Map> generateReplaceInstant(String replaceInstant, Map> instantsToFiles) throws IOException { + Map> partitionToReplacedFileIds = pickFilesToReplace(instantsToFiles); + // generate new fileIds for replace + List newFileIdsToUse = IntStream.range(0, NUM_FILE_IDS_PER_PARTITION).mapToObj(x -> UUID.randomUUID().toString()).collect(Collectors.toList()); + List replacedFiles = addReplaceInstant(metaClient, replaceInstant, + generateDataForInstant(replaceInstant, replaceInstant, false, newFileIdsToUse), + partitionToReplacedFileIds); + instantsToFiles.put(replaceInstant, replacedFiles); + return partitionToReplacedFileIds; + } + + // pick one fileId from each partition to replace and remove it from 'instantsToFiles' + private Map> pickFilesToReplace(Map> instantsToFiles) { + if (instantsToFiles.isEmpty()) { + return Collections.emptyMap(); + } + + String maxInstant = instantsToFiles.keySet().stream().max(Comparator.naturalOrder()).get(); + Map> partitionToFileIdsList = instantsToFiles.get(maxInstant).stream().map(file -> { + int lastPartition = file.lastIndexOf("/"); + return Pair.of(file.substring(0, lastPartition), file.substring(lastPartition + 1)); + }).collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList()))); + return partitions.stream() + .map(p -> Pair.of(p, FSUtils.getFileId(partitionToFileIdsList.get(p).get(0)))) + .collect(Collectors.groupingBy(Pair::getKey, Collectors.mapping(Pair::getValue, Collectors.toList()))); + } + + private HoodieInstant getHoodieReplaceInstant(String timestamp) { + return new HoodieInstant(false, HoodieTimeline.REPLACE_COMMIT_ACTION, timestamp); + } + + private HoodieInstant getHoodieCommitInstant(String timestamp, boolean isDeltaCommit) { + String action = isDeltaCommit ? HoodieTimeline.DELTA_COMMIT_ACTION : HoodieTimeline.COMMIT_ACTION; + return new HoodieInstant(false, action, timestamp); } /** @@ -263,7 +370,7 @@ public void testMultipleTransitions() throws IOException { put("11", Arrays.asList("12", "13", "14")); } }, - instantsToFiles, Collections.singletonList("11")); + instantsToFiles, Collections.singletonList("11"), 0, 0); scheduleCompaction(view2, "20"); instantsToFiles.putAll(testMultipleWriteSteps(view2, Arrays.asList("21", "22"), true, "20", 2)); // Compaction @@ -280,8 +387,8 @@ public void testMultipleTransitions() throws IOException { /* * Case where multiple restores and ingestions happened */ - testRestore(view2, Collections.singletonList("25"), true, new HashMap<>(), Collections.singletonList("24"), "29", true); - testRestore(view2, Collections.singletonList("26"), true, new HashMap<>(), Collections.singletonList("23"), "29", false); + testRestore(view2, Collections.singletonList("25"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("24", true)), "29", true); + testRestore(view2, Collections.singletonList("26"), new HashMap<>(), Collections.singletonList(getHoodieCommitInstant("23", true)), "29", false); instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("27"), true, "20", 2)); scheduleCompaction(view2, "28"); instantsToFiles.putAll(testMultipleWriteSteps(view2, Collections.singletonList("29"), true, "28", 3)); @@ -304,10 +411,11 @@ public void testMultipleTransitions() throws IOException { * Helper to run one or more rounds of cleaning, incrementally syncing the view and then validate. */ private void testCleans(SyncableFileSystemView view, List newCleanerInstants, - Map> instantsToFiles, List cleanedInstants) { + Map> instantsToFiles, List cleanedInstants, int numberOfFilesAddedPerInstant, + int numberOfFilesReplacedPerInstant) { Map> deltaInstantMap = cleanedInstants.stream().map(e -> Pair.of(e, new ArrayList())) .collect(Collectors.toMap(Pair::getKey, Pair::getValue)); - testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants); + testCleans(view, newCleanerInstants, deltaInstantMap, instantsToFiles, cleanedInstants, numberOfFilesAddedPerInstant, numberOfFilesReplacedPerInstant); } /** @@ -321,7 +429,8 @@ private void testCleans(SyncableFileSystemView view, List newCleanerInst */ private void testCleans(SyncableFileSystemView view, List newCleanerInstants, Map> deltaInstantMap, Map> instantsToFiles, - List cleanedInstants) { + List cleanedInstants, int numFilesAddedPerInstant, int numFilesReplacedPerInstant) { + final int netFilesAddedPerInstant = numFilesAddedPerInstant - numFilesReplacedPerInstant; assertEquals(newCleanerInstants.size(), cleanedInstants.size()); long exp = partitions.stream().mapToLong(p1 -> view.getAllFileSlices(p1).count()).findAny().getAsLong(); LOG.info("Initial File Slices :" + exp); @@ -333,7 +442,7 @@ private void testCleans(SyncableFileSystemView view, List newCleanerInst performClean(instant, filesToDelete, newCleanerInstants.get(idx)); - exp -= fileIdsPerPartition.size(); + exp -= fileIdsPerPartition.size() - numFilesReplacedPerInstant; final long expTotalFileSlicesPerPartition = exp; view.sync(); assertTrue(view.getLastInstant().isPresent()); @@ -345,7 +454,8 @@ private void testCleans(SyncableFileSystemView view, List newCleanerInst LOG.info("\tFileSlices :" + view.getAllFileSlices(p).collect(Collectors.toList())); }); - partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count())); + final int instantIdx = newCleanerInstants.size() - idx; + partitions.forEach(p -> assertEquals(fileIdsPerPartition.size() + instantIdx * netFilesAddedPerInstant, view.getLatestFileSlices(p).count())); partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count())); metaClient.reloadActiveTimeline(); @@ -362,22 +472,30 @@ private void testCleans(SyncableFileSystemView view, List newCleanerInst * * @param view Hoodie View * @param newRestoreInstants Restore Instants - * @param isDeltaCommit is Delta Commit ? * @param instantsToFiles List of files associated with each instant * @param rolledBackInstants List of rolled-back instants * @param emptyRestoreInstant Restore instant at which table becomes empty */ - private void testRestore(SyncableFileSystemView view, List newRestoreInstants, boolean isDeltaCommit, - Map> instantsToFiles, List rolledBackInstants, String emptyRestoreInstant, - boolean isRestore) { + private void testRestore(SyncableFileSystemView view, List newRestoreInstants, + Map> instantsToFiles, List rolledBackInstants, String emptyRestoreInstant, + boolean isRestore) { + testRestore(view, newRestoreInstants, instantsToFiles, rolledBackInstants, emptyRestoreInstant, isRestore, 0, 0); + } + + private void testRestore(SyncableFileSystemView view, List newRestoreInstants, + Map> instantsToFiles, List rolledBackInstants, String emptyRestoreInstant, + boolean isRestore, int totalReplacedFileSlicesPerPartition, int totalFilesAddedPerPartitionPerInstant) { assertEquals(newRestoreInstants.size(), rolledBackInstants.size()); long initialFileSlices = partitions.stream().mapToLong(p -> view.getAllFileSlices(p).count()).findAny().getAsLong(); + final int numFileSlicesAddedPerInstant = (totalFilesAddedPerPartitionPerInstant - totalReplacedFileSlicesPerPartition); + final long expectedLatestFileSlices = fileIdsPerPartition.size() + (rolledBackInstants.size()) * numFileSlicesAddedPerInstant; IntStream.range(0, newRestoreInstants.size()).forEach(idx -> { - String instant = rolledBackInstants.get(idx); + HoodieInstant instant = rolledBackInstants.get(idx); try { - performRestore(instant, instantsToFiles.get(instant), newRestoreInstants.get(idx), isRestore); + boolean isDeltaCommit = HoodieTimeline.DELTA_COMMIT_ACTION.equalsIgnoreCase(instant.getAction()); + performRestore(instant, instantsToFiles.get(instant.getTimestamp()), newRestoreInstants.get(idx), isRestore); final long expTotalFileSlicesPerPartition = - isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * fileIdsPerPartition.size()); + isDeltaCommit ? initialFileSlices : initialFileSlices - ((idx + 1) * (fileIdsPerPartition.size() - totalReplacedFileSlicesPerPartition)); view.sync(); assertTrue(view.getLastInstant().isPresent()); LOG.info("Last Instant is :" + view.getLastInstant().get()); @@ -391,7 +509,7 @@ private void testRestore(SyncableFileSystemView view, List newRestoreIns )) { partitions.forEach(p -> assertEquals(0, view.getLatestFileSlices(p).count())); } else { - partitions.forEach(p -> assertEquals(fileIdsPerPartition.size(), view.getLatestFileSlices(p).count())); + partitions.forEach(p -> assertEquals(expectedLatestFileSlices - (idx + 1) * numFileSlicesAddedPerInstant, view.getLatestFileSlices(p).count())); } partitions.forEach(p -> assertEquals(expTotalFileSlicesPerPartition, view.getAllFileSlices(p).count())); @@ -432,28 +550,23 @@ private void performClean(String instant, List files, String cleanInstan * @param files List of files to be deleted as part of rollback * @param rollbackInstant Restore Instant */ - private void performRestore(String instant, List files, String rollbackInstant, + private void performRestore(HoodieInstant instant, List files, String rollbackInstant, boolean isRestore) throws IOException { Map> partititonToFiles = deleteFiles(files); List rollbackStats = partititonToFiles.entrySet().stream().map(e -> new HoodieRollbackStat(e.getKey(), e.getValue(), new ArrayList<>(), new HashMap<>()) ).collect(Collectors.toList()); - List rollbacks = new ArrayList<>(); + List rollbacks = new ArrayList<>(); rollbacks.add(instant); HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.convertRollbackMetadata(rollbackInstant, Option.empty(), rollbacks, rollbackStats); if (isRestore) { - HoodieRestoreMetadata metadata = new HoodieRestoreMetadata(); - List rollbackM = new ArrayList<>(); rollbackM.add(rollbackMetadata); - metadata.setHoodieRestoreMetadata(CollectionUtils.createImmutableMap(rollbackInstant, rollbackM)); - List rollbackInstants = new ArrayList<>(); - rollbackInstants.add(rollbackInstant); - metadata.setInstantsToRollback(rollbackInstants); - metadata.setStartRestoreTime(rollbackInstant); + HoodieRestoreMetadata metadata = TimelineMetadataUtils.convertRestoreMetadata(rollbackInstant, + 100, Collections.singletonList(instant), CollectionUtils.createImmutableMap(rollbackInstant, rollbackM)); HoodieInstant restoreInstant = new HoodieInstant(true, HoodieTimeline.RESTORE_ACTION, rollbackInstant); metaClient.getActiveTimeline().createNewInstant(restoreInstant); @@ -465,6 +578,8 @@ private void performRestore(String instant, List files, String rollbackI new HoodieInstant(true, HoodieTimeline.ROLLBACK_ACTION, rollbackInstant), TimelineMetadataUtils.serializeRollbackMetadata(rollbackMetadata)); } + boolean deleted = metaClient.getFs().delete(new Path(metaClient.getMetaPath(), instant.getFileName()), false); + assertTrue(deleted); } /** @@ -651,10 +766,7 @@ private Map> testMultipleWriteSteps(SyncableFileSystemView private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystemView view2, long expectedTotalFileSlices) { // Timeline check - HoodieTimeline timeline1 = view1.getTimeline(); - HoodieTimeline timeline2 = view2.getTimeline(); assertEquals(view1.getLastInstant(), view2.getLastInstant()); - CollectionUtils.elementsEqual(timeline1.getInstants().iterator(), timeline2.getInstants().iterator()); // View Checks Map fileGroupsMap1 = partitions.stream().flatMap(view1::getAllFileGroups) @@ -702,14 +814,17 @@ private void areViewsConsistent(SyncableFileSystemView view1, SyncableFileSystem assertEquals(ops1, ops2); } - private List addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit, - String baseInstant) throws IOException { - List> writeStats = partitions.stream().flatMap(p -> fileIdsPerPartition.stream().map(f -> { + private List> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit) { + return generateDataForInstant(baseInstant, instant, deltaCommit, fileIdsPerPartition); + } + + private List> generateDataForInstant(String baseInstant, String instant, boolean deltaCommit, List fileIds) { + return partitions.stream().flatMap(p -> fileIds.stream().map(f -> { try { File file = new File(basePath + "/" + p + "/" + (deltaCommit - ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN) - : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f))); + ? FSUtils.makeLogFileName(f, ".log", baseInstant, Integer.parseInt(instant), TEST_WRITE_TOKEN) + : FSUtils.makeDataFileName(instant, TEST_WRITE_TOKEN, f))); file.createNewFile(); HoodieWriteStat w = new HoodieWriteStat(); w.setFileId(f); @@ -719,7 +834,11 @@ private List addInstant(HoodieTableMetaClient metaClient, String instant throw new HoodieException(e); } })).collect(Collectors.toList()); + } + private List addInstant(HoodieTableMetaClient metaClient, String instant, boolean deltaCommit, + String baseInstant) throws IOException { + List> writeStats = generateDataForInstant(baseInstant, instant, deltaCommit); HoodieCommitMetadata metadata = new HoodieCommitMetadata(); writeStats.forEach(e -> metadata.addWriteStat(e.getKey(), e.getValue())); HoodieInstant inflightInstant = new HoodieInstant(true, @@ -735,6 +854,19 @@ private List addInstant(HoodieTableMetaClient metaClient, String instant return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList()); } + private List addReplaceInstant(HoodieTableMetaClient metaClient, String instant, + List> writeStats, + Map> partitionToReplaceFileIds) throws IOException { + HoodieReplaceCommitMetadata replaceCommitMetadata = new HoodieReplaceCommitMetadata(); + writeStats.forEach(e -> replaceCommitMetadata.addWriteStat(e.getKey(), e.getValue())); + replaceCommitMetadata.setPartitionToReplaceFileIds(partitionToReplaceFileIds); + HoodieInstant inflightInstant = new HoodieInstant(true, HoodieTimeline.REPLACE_COMMIT_ACTION, instant); + metaClient.getActiveTimeline().createNewInstant(inflightInstant); + metaClient.getActiveTimeline().saveAsComplete(inflightInstant, + Option.of(replaceCommitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + return writeStats.stream().map(e -> e.getValue().getPath()).collect(Collectors.toList()); + } + @Override protected HoodieTableType getTableType() { return HoodieTableType.MERGE_ON_READ; 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 5da81f3ccad49..16d1ff95001db 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 @@ -19,9 +19,11 @@ package org.apache.hudi.common.testutils; +import org.apache.hadoop.fs.FileSystem; import org.apache.hudi.common.fs.FSUtils; import org.apache.hudi.common.model.HoodieFileFormat; import org.apache.hudi.common.model.HoodiePartitionMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -29,10 +31,9 @@ import org.apache.hudi.common.table.view.TableFileSystemView; import org.apache.hudi.exception.HoodieException; -import org.apache.hadoop.fs.FileSystem; - import java.io.IOException; import java.io.RandomAccessFile; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; @@ -76,6 +77,15 @@ private static void createMetaFile(String basePath, String instantTime, String s } } + private static void createMetaFile(String basePath, String instantTime, String suffix, byte[] content) throws IOException { + Path parentPath = Paths.get(basePath, HoodieTableMetaClient.METAFOLDER_NAME); + Files.createDirectories(parentPath); + Path metaFilePath = parentPath.resolve(instantTime + suffix); + if (Files.notExists(metaFilePath)) { + Files.write(metaFilePath, content); + } + } + public static void createCommit(String basePath, String instantTime) throws IOException { createMetaFile(basePath, instantTime, HoodieTimeline.COMMIT_EXTENSION); } @@ -100,6 +110,18 @@ public static void createInflightDeltaCommit(String basePath, String instantTime createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_DELTA_COMMIT_EXTENSION); } + public static void createReplaceCommit(String basePath, String instantTime, HoodieReplaceCommitMetadata metadata) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REPLACE_COMMIT_EXTENSION, metadata.toJsonString().getBytes(StandardCharsets.UTF_8)); + } + + public static void createRequestedReplaceCommit(String basePath, String instantTime) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.REQUESTED_REPLACE_COMMIT_EXTENSION); + } + + public static void createInflightReplaceCommit(String basePath, String instantTime) throws IOException { + createMetaFile(basePath, instantTime, HoodieTimeline.INFLIGHT_REPLACE_COMMIT_EXTENSION); + } + private static void createAuxiliaryMetaFile(String basePath, String instantTime, String suffix) throws IOException { Path parentPath = Paths.get(basePath, HoodieTableMetaClient.AUXILIARYFOLDER_NAME); Files.createDirectories(parentPath); diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java index 34d59269b3069..bf9728045e530 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestDataGenerator.java @@ -330,30 +330,42 @@ public GenericRecord generateRecordForShortTripSchema(String rowKey, String ride } public static void createCommitFile(String basePath, String instantTime, Configuration configuration) { + HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); + createCommitFile(basePath, instantTime, configuration, commitMetadata); + } + + public static void createCommitFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) { Arrays.asList(HoodieTimeline.makeCommitFileName(instantTime), HoodieTimeline.makeInflightCommitFileName(instantTime), HoodieTimeline.makeRequestedCommitFileName(instantTime)) - .forEach(f -> { - Path commitFile = new Path( - basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f); - FSDataOutputStream os = null; - try { - FileSystem fs = FSUtils.getFs(basePath, configuration); - os = fs.create(commitFile, true); - HoodieCommitMetadata commitMetadata = new HoodieCommitMetadata(); - // Write empty commit metadata - os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); - } catch (IOException ioe) { - throw new HoodieIOException(ioe.getMessage(), ioe); - } finally { - if (null != os) { - try { - os.close(); - } catch (IOException e) { - throw new HoodieIOException(e.getMessage(), e); - } - } - } - }); + .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); + } + + private static void createMetadataFile(String f, String basePath, Configuration configuration, HoodieCommitMetadata commitMetadata) { + Path commitFile = new Path( + basePath + "/" + HoodieTableMetaClient.METAFOLDER_NAME + "/" + f); + FSDataOutputStream os = null; + try { + FileSystem fs = FSUtils.getFs(basePath, configuration); + os = fs.create(commitFile, true); + // Write empty commit metadata + os.writeBytes(new String(commitMetadata.toJsonString().getBytes(StandardCharsets.UTF_8))); + } catch (IOException ioe) { + throw new HoodieIOException(ioe.getMessage(), ioe); + } finally { + if (null != os) { + try { + os.close(); + } catch (IOException e) { + throw new HoodieIOException(e.getMessage(), e); + } + } + } + } + + public static void createReplaceFile(String basePath, String instantTime, Configuration configuration, HoodieCommitMetadata commitMetadata) { + Arrays.asList(HoodieTimeline.makeReplaceFileName(instantTime), HoodieTimeline.makeInflightReplaceFileName(instantTime), + HoodieTimeline.makeRequestedReplaceFileName(instantTime)) + .forEach(f -> createMetadataFile(f, basePath, configuration, commitMetadata)); } public static void createEmptyCleanRequestedFile(String basePath, String instantTime, Configuration configuration) @@ -382,9 +394,9 @@ public static void createCompactionAuxiliaryMetadata(String basePath, HoodieInst new Path(basePath + "/" + HoodieTableMetaClient.AUXILIARYFOLDER_NAME + "/" + instant.getFileName()); FileSystem fs = FSUtils.getFs(basePath, configuration); try (FSDataOutputStream os = fs.create(commitFile, true)) { - HoodieCompactionPlan workload = new HoodieCompactionPlan(); + HoodieCompactionPlan workload = HoodieCompactionPlan.newBuilder().setVersion(1).build(); // Write empty commit metadata - os.writeBytes(new String(TimelineMetadataUtils.serializeCompactionPlan(workload).get(), StandardCharsets.UTF_8)); + os.write(TimelineMetadataUtils.serializeCompactionPlan(workload).get()); } } 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 02ed8e03469c9..06a4aa4b26ca7 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 @@ -19,6 +19,7 @@ package org.apache.hudi.common.testutils; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; import org.apache.hudi.common.model.IOType; import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieTimeline; @@ -49,10 +50,13 @@ import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightCompaction; import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightDeltaCommit; +import static org.apache.hudi.common.testutils.FileCreateUtils.createInflightReplaceCommit; import static org.apache.hudi.common.testutils.FileCreateUtils.createMarkerFile; +import static org.apache.hudi.common.testutils.FileCreateUtils.createReplaceCommit; 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.logFileName; public class HoodieTestTable { @@ -145,6 +149,15 @@ public HoodieTestTable addDeltaCommit(String instantTime) throws Exception { return this; } + public HoodieTestTable addReplaceCommit(String instantTime, HoodieReplaceCommitMetadata metadata) throws Exception { + createRequestedReplaceCommit(basePath, instantTime); + createInflightReplaceCommit(basePath, instantTime); + createReplaceCommit(basePath, instantTime, metadata); + currentInstantTime = instantTime; + metaClient = HoodieTableMetaClient.reload(metaClient); + return this; + } + public HoodieTestTable addRequestedCompaction(String instantTime) throws IOException { createRequestedCompaction(basePath, instantTime); currentInstantTime = instantTime; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java index a1dbe086e5987..6547f807c78f6 100644 --- a/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java +++ b/hudi-common/src/test/java/org/apache/hudi/common/testutils/HoodieTestUtils.java @@ -18,6 +18,21 @@ package org.apache.hudi.common.testutils; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; +import com.esotericsoftware.kryo.serializers.JavaSerializer; +import org.apache.avro.Schema; +import org.apache.avro.generic.GenericRecord; +import org.apache.avro.generic.IndexedRecord; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.util.StringUtils; import org.apache.hudi.avro.HoodieAvroUtils; import org.apache.hudi.avro.model.HoodieActionInstant; import org.apache.hudi.avro.model.HoodieCleanMetadata; @@ -52,22 +67,6 @@ import org.apache.hudi.common.util.collection.Pair; import org.apache.hudi.exception.HoodieIOException; -import com.esotericsoftware.kryo.Kryo; -import com.esotericsoftware.kryo.io.Input; -import com.esotericsoftware.kryo.io.Output; -import com.esotericsoftware.kryo.serializers.JavaSerializer; -import org.apache.avro.Schema; -import org.apache.avro.generic.GenericRecord; -import org.apache.avro.generic.IndexedRecord; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.FSDataOutputStream; -import org.apache.hadoop.fs.FileStatus; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.LocatedFileStatus; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.fs.RemoteIterator; -import org.apache.hadoop.util.StringUtils; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.File; diff --git a/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java new file mode 100644 index 0000000000000..98535e95238e6 --- /dev/null +++ b/hudi-common/src/test/java/org/apache/hudi/common/util/TestCommitUtils.java @@ -0,0 +1,96 @@ +/* + * 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.hudi.common.model.HoodieCommitMetadata; +import org.apache.hudi.common.model.HoodieReplaceCommitMetadata; +import org.apache.hudi.common.model.HoodieWriteStat; +import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_SCHEMA; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class TestCommitUtils { + + @Test + public void testCommitMetadataCreation() { + List writeStats = new ArrayList<>(); + writeStats.add(createWriteStat("p1", "f1")); + writeStats.add(createWriteStat("p2", "f2")); + Map> partitionToReplaceFileIds = new HashMap<>(); + List replacedFileIds = new ArrayList<>(); + replacedFileIds.add("f0"); + partitionToReplaceFileIds.put("p1", replacedFileIds); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds, + Option.empty(), + WriteOperationType.INSERT, + TRIP_SCHEMA, + HoodieTimeline.DELTA_COMMIT_ACTION); + + assertFalse(commitMetadata instanceof HoodieReplaceCommitMetadata); + assertEquals(2, commitMetadata.getPartitionToWriteStats().size()); + assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId()); + assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId()); + assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType()); + assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY)); + } + + @Test + public void testReplaceMetadataCreation() { + List writeStats = new ArrayList<>(); + writeStats.add(createWriteStat("p1", "f1")); + writeStats.add(createWriteStat("p2", "f2")); + + Map> partitionToReplaceFileIds = new HashMap<>(); + List replacedFileIds = new ArrayList<>(); + replacedFileIds.add("f0"); + partitionToReplaceFileIds.put("p1", replacedFileIds); + HoodieCommitMetadata commitMetadata = CommitUtils.buildMetadata(writeStats, partitionToReplaceFileIds, + Option.empty(), + WriteOperationType.INSERT, + TRIP_SCHEMA, + HoodieTimeline.REPLACE_COMMIT_ACTION); + + assertTrue(commitMetadata instanceof HoodieReplaceCommitMetadata); + HoodieReplaceCommitMetadata replaceCommitMetadata = (HoodieReplaceCommitMetadata) commitMetadata; + assertEquals(1, replaceCommitMetadata.getPartitionToReplaceFileIds().size()); + assertEquals("f0", replaceCommitMetadata.getPartitionToReplaceFileIds().get("p1").get(0)); + assertEquals(2, commitMetadata.getPartitionToWriteStats().size()); + assertEquals("f1", commitMetadata.getPartitionToWriteStats().get("p1").get(0).getFileId()); + assertEquals("f2", commitMetadata.getPartitionToWriteStats().get("p2").get(0).getFileId()); + assertEquals(WriteOperationType.INSERT, commitMetadata.getOperationType()); + assertEquals(TRIP_SCHEMA, commitMetadata.getMetadata(HoodieCommitMetadata.SCHEMA_KEY)); + } + + private HoodieWriteStat createWriteStat(String partition, String fileId) { + HoodieWriteStat writeStat1 = new HoodieWriteStat(); + writeStat1.setPartitionPath(partition); + writeStat1.setFileId(fileId); + return writeStat1; + } +} diff --git a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java index 7983aab9000bd..b61d3a6b38188 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/DataSourceUtils.java @@ -18,14 +18,20 @@ package org.apache.hudi; +import org.apache.avro.generic.GenericRecord; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; import org.apache.hudi.client.HoodieReadClient; import org.apache.hudi.client.HoodieWriteClient; -import org.apache.hudi.client.WriteStatus; +import org.apache.hudi.client.HoodieWriteResult; import org.apache.hudi.common.config.TypedProperties; import org.apache.hudi.common.model.HoodieKey; import org.apache.hudi.common.model.HoodieRecord; import org.apache.hudi.common.model.HoodieRecordPayload; +import org.apache.hudi.common.model.HoodieTableType; import org.apache.hudi.common.model.WriteOperationType; +import org.apache.hudi.common.table.timeline.HoodieTimeline; +import org.apache.hudi.common.util.CommitUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.common.util.ReflectionUtils; import org.apache.hudi.common.util.StringUtils; @@ -42,10 +48,6 @@ import org.apache.hudi.keygen.KeyGenerator; import org.apache.hudi.keygen.parser.AbstractHoodieDateTimeParser; import org.apache.hudi.table.BulkInsertPartitioner; - -import org.apache.avro.generic.GenericRecord; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; import org.apache.log4j.LogManager; import org.apache.log4j.Logger; import org.apache.spark.api.java.JavaRDD; @@ -174,25 +176,35 @@ public static HoodieWriteClient createHoodieClient(JavaSparkContext jssc, String return new HoodieWriteClient<>(jssc, createHoodieConfig(schemaStr, basePath, tblName, parameters), true); } - public static JavaRDD doWriteOperation(HoodieWriteClient client, JavaRDD hoodieRecords, - String instantTime, WriteOperationType operation) throws HoodieException { + public static String getCommitActionType(WriteOperationType operation, HoodieTableType tableType) { + if (operation == WriteOperationType.INSERT_OVERWRITE) { + return HoodieTimeline.REPLACE_COMMIT_ACTION; + } else { + return CommitUtils.getCommitActionType(tableType); + } + } + + public static HoodieWriteResult doWriteOperation(HoodieWriteClient client, JavaRDD hoodieRecords, + String instantTime, WriteOperationType operation) throws HoodieException { switch (operation) { case BULK_INSERT: Option userDefinedBulkInsertPartitioner = createUserDefinedBulkInsertPartitioner(client.getConfig()); - return client.bulkInsert(hoodieRecords, instantTime, userDefinedBulkInsertPartitioner); + return new HoodieWriteResult(client.bulkInsert(hoodieRecords, instantTime, userDefinedBulkInsertPartitioner)); case INSERT: - return client.insert(hoodieRecords, instantTime); + return new HoodieWriteResult(client.insert(hoodieRecords, instantTime)); case UPSERT: - return client.upsert(hoodieRecords, instantTime); + return new HoodieWriteResult(client.upsert(hoodieRecords, instantTime)); + case INSERT_OVERWRITE: + return client.insertOverwrite(hoodieRecords, instantTime); default: throw new HoodieException("Not a valid operation type for doWriteOperation: " + operation.toString()); } } - public static JavaRDD doDeleteOperation(HoodieWriteClient client, JavaRDD hoodieKeys, + public static HoodieWriteResult doDeleteOperation(HoodieWriteClient client, JavaRDD hoodieKeys, String instantTime) { - return client.delete(hoodieKeys, instantTime); + return new HoodieWriteResult(client.delete(hoodieKeys, instantTime)); } public static HoodieRecord createHoodieRecord(GenericRecord gr, Comparable orderingVal, HoodieKey hKey, diff --git a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java b/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java index 8936e03df8778..febdf190c40b4 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java +++ b/hudi-spark/src/main/java/org/apache/hudi/QuickstartUtils.java @@ -73,6 +73,10 @@ public DataGenerator() { this(DEFAULT_PARTITION_PATHS, new HashMap<>()); } + public DataGenerator(String[] partitionPaths) { + this(partitionPaths, new HashMap<>()); + } + private DataGenerator(String[] partitionPaths, Map keyPartitionMap) { this.partitionPaths = Arrays.copyOf(partitionPaths, partitionPaths.length); this.existingKeys = keyPartitionMap; diff --git a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java index 6e67dc9f2ac85..a4bfd59101417 100644 --- a/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java +++ b/hudi-spark/src/main/java/org/apache/hudi/internal/HoodieDataSourceInternalWriter.java @@ -18,11 +18,8 @@ package org.apache.hudi.internal; -import java.util.Arrays; -import java.util.List; -import java.util.stream.Collectors; - import org.apache.hadoop.conf.Configuration; +import org.apache.hudi.DataSourceUtils; import org.apache.hudi.client.HoodieWriteClient; import org.apache.hudi.common.model.HoodieWriteStat; import org.apache.hudi.common.model.WriteOperationType; @@ -44,6 +41,10 @@ import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; import org.apache.spark.sql.types.StructType; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Collectors; + /** * Implementation of {@link DataSourceWriter} for datasource "hudi.internal" to be used in datasource implementation * of bulk insert. @@ -102,7 +103,8 @@ public void commit(WriterCommitMessage[] messages) { .flatMap(m -> m.getWriteStatuses().stream().map(m2 -> m2.getStat())).collect(Collectors.toList()); try { - writeClient.commitStats(instantTime, writeStatList, Option.empty()); + writeClient.commitStats(instantTime, writeStatList, Option.empty(), + DataSourceUtils.getCommitActionType(operationType, metaClient.getTableType())); } catch (Exception ioe) { throw new HoodieException(ioe.getMessage(), ioe); } finally { diff --git a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala index e10bb677ad60b..e153e92496d9f 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/DataSourceOptions.scala @@ -142,6 +142,7 @@ object DataSourceWriteOptions { val UPSERT_OPERATION_OPT_VAL = WriteOperationType.UPSERT.value val DELETE_OPERATION_OPT_VAL = WriteOperationType.DELETE.value val BOOTSTRAP_OPERATION_OPT_VAL = WriteOperationType.BOOTSTRAP.value + val INSERT_OVERWRITE_OPERATION_OPT_VAL = WriteOperationType.INSERT_OVERWRITE.value val DEFAULT_OPERATION_OPT_VAL = UPSERT_OPERATION_OPT_VAL /** diff --git a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala index 8cab7c16e6d99..df7960e5b9cc1 100644 --- a/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala +++ b/hudi-spark/src/main/scala/org/apache/hudi/HoodieSparkSqlWriter.scala @@ -27,13 +27,10 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.conf.HiveConf import org.apache.hudi.DataSourceWriteOptions._ import org.apache.hudi.avro.HoodieAvroUtils -import org.apache.hudi.client.{HoodieWriteClient, WriteStatus} +import org.apache.hudi.client.{HoodieWriteClient, HoodieWriteResult} import org.apache.hudi.common.config.TypedProperties -import org.apache.hudi.common.model.HoodieRecordPayload -import org.apache.hudi.common.model.HoodieTableType -import org.apache.hudi.common.model.WriteOperationType -import org.apache.hudi.common.table.HoodieTableConfig -import org.apache.hudi.common.table.HoodieTableMetaClient +import org.apache.hudi.common.model.{HoodieRecordPayload, HoodieTableType, WriteOperationType} +import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient} import org.apache.hudi.common.table.timeline.HoodieActiveTimeline import org.apache.hudi.common.util.ReflectionUtils import org.apache.hudi.config.HoodieBootstrapConfig.{BOOTSTRAP_BASE_PATH_PROP, BOOTSTRAP_INDEX_CLASS_PROP, DEFAULT_BOOTSTRAP_INDEX_CLASS} @@ -44,7 +41,7 @@ import org.apache.hudi.internal.HoodieDataSourceInternalWriter import org.apache.hudi.sync.common.AbstractSyncTool import org.apache.log4j.LogManager import org.apache.spark.SparkContext -import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} @@ -80,7 +77,7 @@ private[hudi] object HoodieSparkSqlWriter { case Some(ser) if ser.equals("org.apache.spark.serializer.KryoSerializer") => case _ => throw new HoodieException("hoodie only support org.apache.spark.serializer.KryoSerializer as spark.serializer") } - val tableType = parameters(TABLE_TYPE_OPT_KEY) + val tableType = HoodieTableType.valueOf(parameters(TABLE_TYPE_OPT_KEY)) var operation = WriteOperationType.fromValue(parameters(OPERATION_OPT_KEY)) // It does not make sense to allow upsert() operation if INSERT_DROP_DUPS_OPT_KEY is true // Auto-correct the operation to "insert" if OPERATION_OPT_KEY is set to "upsert" wrongly @@ -113,11 +110,13 @@ private[hudi] object HoodieSparkSqlWriter { val archiveLogFolder = parameters.getOrElse( HoodieTableConfig.HOODIE_ARCHIVELOG_FOLDER_PROP_NAME, "archived") val tableMetaClient = HoodieTableMetaClient.initTableType(sparkContext.hadoopConfiguration, path.get, - HoodieTableType.valueOf(tableType), tblName, archiveLogFolder, parameters(PAYLOAD_CLASS_OPT_KEY), + tableType, tblName, archiveLogFolder, parameters(PAYLOAD_CLASS_OPT_KEY), null.asInstanceOf[String]) tableConfig = tableMetaClient.getTableConfig } + val commitActionType = DataSourceUtils.getCommitActionType(operation, tableConfig.getTableType) + // short-circuit if bulk_insert via row is enabled. // scalastyle:off if (parameters(ENABLE_ROW_WRITER_OPT_KEY).toBoolean) { @@ -127,7 +126,7 @@ private[hudi] object HoodieSparkSqlWriter { } // scalastyle:on - val (writeStatuses, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) = + val (writeResult, writeClient: HoodieWriteClient[HoodieRecordPayload[Nothing]]) = if (operation != WriteOperationType.DELETE) { // register classes & schemas val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(tblName) @@ -169,9 +168,9 @@ private[hudi] object HoodieSparkSqlWriter { log.info("new batch has no new records, skipping...") (true, common.util.Option.empty()) } - client.startCommitWithTime(instantTime) - val writeStatuses = DataSourceUtils.doWriteOperation(client, hoodieRecords, instantTime, operation) - (writeStatuses, client) + client.startCommitWithTime(instantTime, commitActionType) + val writeResult = DataSourceUtils.doWriteOperation(client, hoodieRecords, instantTime, operation) + (writeResult, client) } else { val structName = s"${tblName}_record" val nameSpace = s"hoodie.${tblName}" @@ -198,15 +197,15 @@ private[hudi] object HoodieSparkSqlWriter { } // Issue deletes - client.startCommitWithTime(instantTime) + client.startCommitWithTime(instantTime, commitActionType) val writeStatuses = DataSourceUtils.doDeleteOperation(client, hoodieKeysToDelete, instantTime) (writeStatuses, client) } // Check for errors and commit the write. val (writeSuccessful, compactionInstant) = - commitAndPerformPostOperations(writeStatuses, parameters, writeClient, tableConfig, instantTime, basePath, - operation, jsc) + commitAndPerformPostOperations(writeResult, parameters, writeClient, tableConfig, jsc, + TableInstantInfo(basePath, instantTime, commitActionType, operation)) (writeSuccessful, common.util.Option.ofNullable(instantTime), compactionInstant, writeClient, tableConfig) } } @@ -383,31 +382,34 @@ private[hudi] object HoodieSparkSqlWriter { metaSyncSuccess } - private def commitAndPerformPostOperations(writeStatuses: JavaRDD[WriteStatus], + /** + * Group all table/action specific information into a case class. + */ + case class TableInstantInfo(basePath: Path, instantTime: String, commitActionType: String, operation: WriteOperationType) + + private def commitAndPerformPostOperations(writeResult: HoodieWriteResult, parameters: Map[String, String], client: HoodieWriteClient[HoodieRecordPayload[Nothing]], tableConfig: HoodieTableConfig, - instantTime: String, - basePath: Path, - operation: WriteOperationType, - jsc: JavaSparkContext): (Boolean, common.util.Option[java.lang.String]) = { - val errorCount = writeStatuses.rdd.filter(ws => ws.hasErrors).count() + jsc: JavaSparkContext, + tableInstantInfo: TableInstantInfo + ): (Boolean, common.util.Option[java.lang.String]) = { + val errorCount = writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors).count() if (errorCount == 0) { log.info("No errors. Proceeding to commit the write.") val metaMap = parameters.filter(kv => kv._1.startsWith(parameters(COMMIT_METADATA_KEYPREFIX_OPT_KEY))) - val commitSuccess = if (metaMap.isEmpty) { - client.commit(instantTime, writeStatuses) - } else { - client.commit(instantTime, writeStatuses, - common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap)))) - } + val commitSuccess = + client.commit(tableInstantInfo.instantTime, writeResult.getWriteStatuses, + common.util.Option.of(new util.HashMap[String, String](mapAsJavaMap(metaMap))), + tableInstantInfo.commitActionType, + writeResult.getPartitionToReplaceFileIds) if (commitSuccess) { - log.info("Commit " + instantTime + " successful!") + log.info("Commit " + tableInstantInfo.instantTime + " successful!") } else { - log.info("Commit " + instantTime + " failed!") + log.info("Commit " + tableInstantInfo.instantTime + " failed!") } val asyncCompactionEnabled = isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration()) @@ -419,7 +421,7 @@ private[hudi] object HoodieSparkSqlWriter { } log.info(s"Compaction Scheduled is $compactionInstant") - val metaSyncSuccess = metaSync(parameters, basePath, jsc.hadoopConfiguration()) + val metaSyncSuccess = metaSync(parameters, tableInstantInfo.basePath, jsc.hadoopConfiguration()) log.info(s"Is Async Compaction Enabled ? $asyncCompactionEnabled") if (!asyncCompactionEnabled) { @@ -427,10 +429,10 @@ private[hudi] object HoodieSparkSqlWriter { } (commitSuccess && metaSyncSuccess, compactionInstant) } else { - log.error(s"${operation.toString} failed with $errorCount errors :") + log.error(s"${tableInstantInfo.operation} failed with $errorCount errors :") if (log.isTraceEnabled) { log.trace("Printing out the top 100 errors") - writeStatuses.rdd.filter(ws => ws.hasErrors) + writeResult.getWriteStatuses.rdd.filter(ws => ws.hasErrors) .take(100) .foreach(ws => { log.trace("Global error :", ws.getGlobalError) diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java index 683eb0658ced6..a4bb52c8612df 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/FileSystemViewHandler.java @@ -289,6 +289,14 @@ private void registerFileSlicesAPI() { .refreshTable(ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow()); writeValueAsString(ctx, success); }, false)); + + app.get(RemoteHoodieTableFileSystemView.ALL_REPLACED_FILEGROUPS_BEFORE_OR_ON, new ViewHandler(ctx -> { + List dtos = sliceHandler.getReplacedFileGroupsBeforeOrOn( + ctx.validatedQueryParam(RemoteHoodieTableFileSystemView.BASEPATH_PARAM).getOrThrow(), + ctx.queryParam(RemoteHoodieTableFileSystemView.MAX_INSTANT_PARAM,""), + ctx.queryParam(RemoteHoodieTableFileSystemView.PARTITION_PARAM,"")); + writeValueAsString(ctx, dtos); + }, true)); } private static boolean isRefreshCheckDisabledInQuery(Context ctxt) { diff --git a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java index c6c3afcebed25..56cb7a870c03d 100644 --- a/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java +++ b/hudi-timeline-service/src/main/java/org/apache/hudi/timeline/service/handlers/FileSliceHandler.java @@ -18,13 +18,12 @@ package org.apache.hudi.timeline.service.handlers; +import org.apache.hadoop.conf.Configuration; import org.apache.hudi.common.table.timeline.dto.CompactionOpDTO; import org.apache.hudi.common.table.timeline.dto.FileGroupDTO; import org.apache.hudi.common.table.timeline.dto.FileSliceDTO; import org.apache.hudi.common.table.view.FileSystemViewManager; -import org.apache.hadoop.conf.Configuration; - import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -89,6 +88,11 @@ public List getAllFileGroups(String basePath, String partitionPath .collect(Collectors.toList()); } + public List getReplacedFileGroupsBeforeOrOn(String basePath, String maxCommitTime, String partitionPath) { + return viewManager.getFileSystemView(basePath).getReplacedFileGroupsBeforeOrOn(maxCommitTime, partitionPath).map(FileGroupDTO::fromFileGroup) + .collect(Collectors.toList()); + } + public boolean refreshTable(String basePath) { viewManager.clearFileSystemView(basePath); return true;