1
0

[HUDI-1072] Introduce REPLACE top level action. Implement insert_overwrite operation on top of replace action (#2048)

This commit is contained in:
satishkotha
2020-09-29 17:04:25 -07:00
committed by GitHub
parent 32c9cad52c
commit a99e93bed5
60 changed files with 2129 additions and 380 deletions

View File

@@ -32,6 +32,7 @@ import org.apache.hudi.common.table.HoodieTableVersion;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.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.index.HoodieIndex;
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 org.apache.spark.api.java.JavaSparkContext;
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 abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload> e
}
/**
*
* Commit changes performed at the given instantTime marker.
*/
public boolean commit(String instantTime, JavaRDD<WriteStatus> writeStatuses,
Option<Map<String, String>> extraMetadata) {
List<HoodieWriteStat> stats = writeStatuses.map(WriteStatus::getStat).collect();
return commitStats(instantTime, stats, extraMetadata);
}
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata) {
LOG.info("Committing " + instantTime);
Option<Map<String, String>> 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<WriteStatus> writeStatuses,
Option<Map<String, String>> extraMetadata, String commitActionType, Map<String, List<String>> partitionToReplacedFileIds) {
List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
return commitStats(instantTime, writeStats, extraMetadata, commitActionType, partitionToReplacedFileIds);
}
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
String commitActionType) {
return commitStats(instantTime, stats, extraMetadata, commitActionType, Collections.emptyMap());
}
public boolean commitStats(String instantTime, List<HoodieWriteStat> stats, Option<Map<String, String>> extraMetadata,
String commitActionType, Map<String, List<String>> partitionToReplaceFileIds) {
LOG.info("Committing " + instantTime + " action " + commitActionType);
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> 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,

View File

@@ -320,6 +320,22 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
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<HoodieRecord<T>> records, final String instantTime) {
HoodieTable<T> 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 class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
rollbackPendingCommits();
}
String instantTime = HoodieActiveTimeline.createNewInstantTime();
startCommit(instantTime);
startCommitWithTime(instantTime);
return instantTime;
}
@@ -586,24 +602,39 @@ public class HoodieWriteClient<T extends HoodieRecordPayload> extends AbstractHo
* @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));
}

View File

@@ -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<WriteStatus> writeStatuses;
private Map<String, List<String>> partitionToReplaceFileIds;
public HoodieWriteResult(JavaRDD<WriteStatus> writeStatuses) {
this(writeStatuses, Collections.emptyMap());
}
public HoodieWriteResult(JavaRDD<WriteStatus> writeStatuses, Map<String, List<String>> partitionToReplaceFileIds) {
this.writeStatuses = writeStatuses;
this.partitionToReplaceFileIds = partitionToReplaceFileIds;
}
public JavaRDD<WriteStatus> getWriteStatuses() {
return this.writeStatuses;
}
public void setWriteStatuses(final JavaRDD<WriteStatus> writeStatuses) {
this.writeStatuses = writeStatuses;
}
public Map<String, List<String>> getPartitionToReplaceFileIds() {
return this.partitionToReplaceFileIds;
}
public void setPartitionToReplaceFileIds(final Map<String, List<String>> partitionToReplaceFileIds) {
this.partitionToReplaceFileIds = partitionToReplaceFileIds;
}
@Override
public String toString() {
return "HoodieWriteResult{"
+ "writeStatuses=" + writeStatuses
+ ", partitionToReplaceFileIds=" + partitionToReplaceFileIds
+ '}';
}
}

View File

@@ -46,6 +46,7 @@ import org.apache.hudi.table.action.commit.BulkInsertCommitActionExecutor;
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 class HoodieCopyOnWriteTable<T extends HoodieRecordPayload> extends Hoodi
this, instantTime, preppedRecords, userDefinedBulkInsertPartitioner).execute();
}
@Override
public HoodieWriteMetadata insertOverwrite(JavaSparkContext jsc, String instantTime,
JavaRDD<HoodieRecord<T>> records) {
return new InsertOverwriteCommitActionExecutor<>(jsc, config, this, instantTime, records).execute();
}
@Override
public Option<HoodieCompactionPlan> scheduleCompaction(JavaSparkContext jsc, String instantTime, Option<Map<String, String>> extraMetadata) {
throw new HoodieNotSupportedException("Compaction is not supported on a CopyOnWrite table");

View File

@@ -213,6 +213,12 @@ public abstract class HoodieTable<T extends HoodieRecordPayload> implements Seri
public abstract HoodieWriteMetadata bulkInsertPrepped(JavaSparkContext jsc, String instantTime,
JavaRDD<HoodieRecord<T>> preppedRecords, Option<BulkInsertPartitioner> 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<HoodieRecord<T>> records);
public HoodieWriteConfig getConfig() {
return config;
}

View File

@@ -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.avro.model.HoodieSavepointMetadata;
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.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.TimelineMetadataUtils;
import org.apache.hudi.common.table.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 class HoodieTimelineArchiveLog {
LOG.info("Wrapper schema " + wrapperSchema.toString());
List<IndexedRecord> 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 @@ public class HoodieTimelineArchiveLog {
}
}
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<String> replacedPartitions = getReplacedPartitions(instant);
return ReplaceArchivalHelper.deleteReplacedFileGroups(jsc, metaClient, fileSystemView, instant, replacedPartitions);
}
private List<String> 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<IndexedRecord> records) throws Exception {
if (records.size() > 0) {
Map<HeaderMetadataType, String> header = new HashMap<>();
@@ -334,6 +363,13 @@ public class HoodieTimelineArchiveLog {
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));

View File

@@ -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<String> replacedPartitions) {
JavaRDD<String> partitions = jsc.parallelize(replacedPartitions, replacedPartitions.size());
return partitions.map(partition -> {
Stream<FileSlice> 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;
}
}
}

View File

@@ -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<List<HoodieWriteStat>> writeStats = Option.empty();
private Option<Duration> indexUpdateDuration = Option.empty();
private Option<Duration> finalizeDuration = Option.empty();
private Option<Map<String, List<String>>> partitionToReplaceFileIds = Option.empty();
public HoodieWriteMetadata() {
}
@@ -101,4 +103,12 @@ public class HoodieWriteMetadata {
public void setIndexLookupDuration(Duration indexLookupDuration) {
this.indexLookupDuration = Option.ofNullable(indexLookupDuration);
}
public Map<String, List<String>> getPartitionToReplaceFileIds() {
return partitionToReplaceFileIds.orElse(Collections.emptyMap());
}
public void setPartitionToReplaceFileIds(Map<String, List<String>> partitionToReplaceFileIds) {
this.partitionToReplaceFileIds = Option.ofNullable(partitionToReplaceFileIds);
}
}

View File

@@ -30,6 +30,7 @@ import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.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.WorkloadProfile;
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.JavaPairRDD;
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<T extends HoodieRecordPayload<T>, R>
extends BaseActionExecutor<R> {
@@ -140,7 +140,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>,
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 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>,
}
}
private Partitioner getPartitioner(WorkloadProfile profile) {
protected Partitioner getPartitioner(WorkloadProfile profile) {
if (WriteOperationType.isChangingRecords(operationType)) {
return getUpsertPartitioner(profile);
} else {
@@ -191,6 +191,7 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>,
(HoodieTable<T>)table);
result.setIndexUpdateDuration(Duration.between(indexStartTime, Instant.now()));
result.setWriteStatuses(statuses);
result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(statuses));
commitOnAutoCommit(result);
}
@@ -203,42 +204,40 @@ public abstract class BaseCommitActionExecutor<T extends HoodieRecordPayload<T>,
}
}
protected String getCommitActionType() {
return table.getMetaClient().getCommitActionType();
}
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result) {
commit(extraMetadata, result, result.getWriteStatuses().map(WriteStatus::getStat).collect());
}
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result, List<HoodieWriteStat> stats) {
String actionType = table.getMetaClient().getCommitActionType();
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata result, List<HoodieWriteStat> writeStats) {
String actionType = getCommitActionType();
LOG.info("Committing " + instantTime + ", action Type " + actionType);
// Create a Hoodie table which encapsulated the commits and files visible
HoodieTable<T> 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<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
return Collections.emptyMap();
}
/**

View File

@@ -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<T extends HoodieRecordPayload<T>>
extends CommitActionExecutor<T> {
private static final Logger LOG = LogManager.getLogger(InsertOverwriteCommitActionExecutor.class);
private final JavaRDD<HoodieRecord<T>> inputRecordsRDD;
public InsertOverwriteCommitActionExecutor(JavaSparkContext jsc,
HoodieWriteConfig config, HoodieTable table,
String instantTime, JavaRDD<HoodieRecord<T>> inputRecordsRDD) {
super(jsc, config, table, instantTime, WriteOperationType.INSERT_OVERWRITE);
this.inputRecordsRDD = inputRecordsRDD;
}
@Override
public HoodieWriteMetadata execute() {
return WriteHelper.write(instantTime, inputRecordsRDD, jsc, (HoodieTable<T>) 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<String, List<String>> getPartitionToReplacedFileIds(JavaRDD<WriteStatus> writeStatuses) {
return writeStatuses.map(status -> status.getStat().getPartitionPath()).distinct().mapToPair(partitionPath ->
new Tuple2<>(partitionPath, getAllExistingFileIds(partitionPath))).collectAsMap();
}
private List<String> 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());
}
}

View File

@@ -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<T extends HoodieRecordPayload<T>> extends UpsertPartitioner {
private static final Logger LOG = LogManager.getLogger(InsertOverwritePartitioner.class);
public InsertOverwritePartitioner(WorkloadProfile profile, JavaSparkContext jsc, HoodieTable<T> table,
HoodieWriteConfig config) {
super(profile, jsc, table, config);
}
/**
* Returns a list of small files in the given partition path.
*/
protected List<SmallFile> getSmallFiles(String partitionPath) {
// for overwrite, we ignore all existing files. So dont consider any file to be smallFiles
return Collections.emptyList();
}
}

View File

@@ -76,7 +76,7 @@ public abstract class BaseRestoreActionExecutor extends BaseActionExecutor<Hoodi
try {
return finishRestore(instantToMetadata,
instantsToRollback.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toList()),
instantsToRollback,
restoreTimer.endTimer()
);
} catch (IOException io) {
@@ -87,7 +87,7 @@ public abstract class BaseRestoreActionExecutor extends BaseActionExecutor<Hoodi
protected abstract HoodieRollbackMetadata rollbackInstant(HoodieInstant rollbackInstant);
private HoodieRestoreMetadata finishRestore(Map<String, List<HoodieRollbackMetadata>> instantToMetadata,
List<String> instantsRolledBack,
List<HoodieInstant> instantsRolledBack,
long durationInMs) throws IOException {
HoodieRestoreMetadata restoreMetadata = TimelineMetadataUtils.convertRestoreMetadata(

View File

@@ -50,7 +50,8 @@ public class CopyOnWriteRestoreActionExecutor extends BaseRestoreActionExecutor
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();

View File

@@ -54,6 +54,7 @@ public class MergeOnReadRestoreActionExecutor extends BaseRestoreActionExecutor
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

View File

@@ -109,7 +109,7 @@ public abstract class BaseRollbackActionExecutor extends BaseActionExecutor<Hood
HoodieRollbackMetadata rollbackMetadata = TimelineMetadataUtils.convertRollbackMetadata(
instantTime,
Option.of(rollbackTimer.endTimer()),
Collections.singletonList(instantToRollback.getTimestamp()),
Collections.singletonList(instantToRollback),
stats);
if (!skipTimelinePublish) {
finishRollback(rollbackMetadata);

View File

@@ -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.table.action.commit.WriteHelper;
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 class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
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<HoodieRecord> inserts1 = dataGen.generateInserts(commitTime1, batch1RecordsCount);
JavaRDD<HoodieRecord> insertRecordsRDD1 = jsc.parallelize(inserts1, 2);
List<WriteStatus> statuses = client.upsert(insertRecordsRDD1, commitTime1).collect();
assertNoWriteErrors(statuses);
Set<String> 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<HoodieRecord> inserts2 = dataGen.generateInserts(commitTime2, batch2RecordsCount);
List<HoodieRecord> insertsAndUpdates2 = new ArrayList<>();
insertsAndUpdates2.addAll(inserts2);
JavaRDD<HoodieRecord> 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<HoodieRecord> expectedRecords, List<WriteStatus> allStatus) {
List<GenericRecord> records = new ArrayList<>();
for (WriteStatus status : allStatus) {
Path filePath = new Path(basePath, status.getStat().getPath());
records.addAll(ParquetUtils.readAvroRecords(jsc.hadoopConfiguration(), filePath));
}
Set<String> 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<Set<String>, List<HoodieRecord>> testUpdates(String instantTime, HoodieWriteClient client,
int sizeToInsertAndUpdate, int expectedTotalRecords)
throws IOException {

View File

@@ -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;
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 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
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<String> 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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
.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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
.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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
.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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
.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<HoodieInstant> 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 class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
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);
}
}