1
0

[HUDI-2285][HUDI-2476] Metadata table synchronous design. Rebased and Squashed from pull/3426 (#3590)

* [HUDI-2285] Adding Synchronous updates to metadata before completion of commits in data timelime.

- This patch adds synchronous updates to metadata table. In other words, every write is first committed to metadata table followed by data table. While reading metadata table, we ignore any delta commits that are present only in metadata table and not in data table timeline.
- Compaction of metadata table is fenced by the condition that we trigger compaction only when there are no inflight requests in datatable. This ensures that all base files in metadata table is always in sync with data table(w/o any holes) and only there could be some extra invalid commits among delta log files in metadata table.
- Due to this, archival of data table also fences itself up until compacted instant in metadata table.
All writes to metadata table happens within the datatable lock. So, metadata table works in one writer mode only. This might be tough to loosen since all writers write to same FILES partition and so, will result in a conflict anyways.
- As part of this, have added acquiring locks in data table for those operations which were not before while committing (rollback, clean, compaction, cluster). To note, we were not doing any conflict resolution. All we are doing here is to commit by taking a lock. So that all writes to metadata table is always a single writer. 
- Also added building block to add buckets for partitions, which will be leveraged by other indexes like record level index, etc. For now, FILES partition has only one bucket. In general, any number of buckets per partition is allowed and each partition has a fixed fileId prefix with incremental suffix for each bucket within each partition.
Have fixed [HUDI-2476]. This fix is about retrying a failed compaction if it succeeded in metadata for first time, but failed w/ data table.
- Enabling metadata table by default.
- Adding more tests for metadata table

Co-authored-by: Prashant Wason <pwason@uber.com>
This commit is contained in:
Sivabalan Narayanan
2021-10-06 00:17:52 -04:00
committed by GitHub
parent 46808dcb1f
commit 5f32162a2f
101 changed files with 3329 additions and 2069 deletions

View File

@@ -42,7 +42,6 @@ import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieClusteringException;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.SparkHoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
@@ -79,7 +78,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
private static final Logger LOG = LogManager.getLogger(SparkRDDWriteClient.class);
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig clientConfig) {
super(context, clientConfig);
this(context, clientConfig, Option.empty());
}
@Deprecated
@@ -96,6 +95,11 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig,
Option<EmbeddedTimelineService> timelineService) {
super(context, writeConfig, timelineService);
if (config.isMetadataTableEnabled()) {
// If the metadata table does not exist, it should be bootstrapped here
// TODO: Check if we can remove this requirement - auto bootstrap on commit
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context);
}
}
/**
@@ -299,12 +303,13 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
String compactionCommitTime) {
this.context.setJobStatus(this.getClass().getSimpleName(), "Collect compaction write status and commit compaction");
List<HoodieWriteStat> writeStats = writeStatuses.map(WriteStatus::getStat).collect();
writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, compactionCommitTime));
// commit to data table after committing to metadata table.
finalizeWrite(table, compactionCommitTime, writeStats);
LOG.info("Committing Compaction " + compactionCommitTime + ". Finished with result " + metadata);
SparkCompactHelpers.newInstance().completeInflightCompaction(table, compactionCommitTime, metadata);
WriteMarkersFactory.get(config.getMarkersType(), table, compactionCommitTime)
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
if (compactionTimer != null) {
long durationInMs = metrics.getDurationInMs(compactionTimer.stop());
try {
@@ -320,7 +325,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
protected JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context, config.isMetadataTableEnabled());
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context, true);
preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient());
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
@@ -369,6 +374,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
throw new HoodieClusteringException("Clustering failed to write to files:"
+ writeStats.stream().filter(s -> s.getTotalWriteErrors() > 0L).map(s -> s.getFileId()).collect(Collectors.joining(",")));
}
writeTableMetadata(table, metadata, new HoodieInstant(HoodieInstant.State.INFLIGHT, HoodieTimeline.REPLACE_COMMIT_ACTION, clusteringCommitTime));
finalizeWrite(table, clusteringCommitTime, writeStats);
try {
LOG.info("Committing Clustering " + clusteringCommitTime + ". Finished with result " + metadata);
@@ -376,7 +382,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTimeline.getReplaceCommitInflightInstant(clusteringCommitTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
} catch (IOException e) {
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
throw new HoodieClusteringException("unable to transition clustering inflight to complete: " + clusteringCommitTime, e);
}
WriteMarkersFactory.get(config.getMarkersType(), table, clusteringCommitTime)
.quietDeleteMarkerDir(context, config.getMarkersDeleteParallelism());
@@ -393,6 +399,18 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
LOG.info("Clustering successfully on commit " + clusteringCommitTime);
}
private void writeTableMetadata(HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table, HoodieCommitMetadata commitMetadata,
HoodieInstant hoodieInstant) {
try {
this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty());
// Do not do any conflict resolution here as we do with regular writes. We take the lock here to ensure all writes to metadata table happens within a
// single lock (single writer). Because more than one write to metadata table will result in conflicts since all of them updates the same partition.
table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp()));
} finally {
this.txnManager.endTransaction();
}
}
@Override
protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
HoodieTableMetaClient metaClient = createMetaClient(true);
@@ -445,24 +463,10 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
} else {
writeTimer = metrics.getDeltaCommitCtx();
}
table.getHoodieView().sync();
return table;
}
@Override
public void syncTableMetadata() {
if (!config.getMetadataConfig().enableSync()) {
LOG.info("Metadata table sync is disabled in the config.");
return;
}
// Open up the metadata table again, for syncing
try (HoodieTableMetadataWriter writer = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
LOG.info("Successfully synced to metadata table");
} catch (Exception e) {
throw new HoodieMetadataException("Error syncing to metadata table.", e);
}
}
@Override
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
// Create a Hoodie table after startTxn which encapsulated the commits and files visible.
@@ -470,6 +474,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable table = createTable(config, hadoopConf);
TransactionUtils.resolveWriteConflictIfAny(table, this.txnManager.getCurrentTransactionOwner(),
Option.of(metadata), config, txnManager.getLastCompletedTransactionOwner());
table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, instantTime));
}
@Override

View File

@@ -24,20 +24,15 @@ import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieLogFile;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordLocation;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.metrics.DistributedRegistry;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.conf.Configuration;
import org.apache.log4j.LogManager;
@@ -46,8 +41,8 @@ import org.apache.spark.api.java.JavaRDD;
import org.apache.spark.api.java.JavaSparkContext;
import java.io.IOException;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetadataWriter {
@@ -78,7 +73,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
}
@Override
protected void initialize(HoodieEngineContext engineContext, HoodieTableMetaClient datasetMetaClient) {
protected void initialize(HoodieEngineContext engineContext) {
try {
metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> {
if (registry instanceof DistributedRegistry) {
@@ -88,7 +83,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
});
if (enabled) {
bootstrapIfNeeded(engineContext, datasetMetaClient);
bootstrapIfNeeded(engineContext, dataMetaClient);
}
} catch (IOException e) {
LOG.error("Failed to initialize metadata table. Disabling the writer.", e);
@@ -99,83 +94,93 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
@Override
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName);
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
try (SparkRDDWriteClient writeClient = new SparkRDDWriteClient(engineContext, metadataWriteConfig, true)) {
writeClient.startCommitWithTime(instantTime);
if (!metadataMetaClient.getActiveTimeline().filterCompletedInstants().containsInstant(instantTime)) {
// if this is a new commit being applied to metadata for the first time
writeClient.startCommitWithTime(instantTime);
} else {
// this code path refers to a re-attempted commit that got committed to metadata table, but failed in datatable.
// for eg, lets say compaction c1 on 1st attempt succeeded in metadata table and failed before committing to datatable.
// when retried again, data table will first rollback pending compaction. these will be applied to metadata table, but all changes
// are upserts to metadata table and so only a new delta commit will be created.
// once rollback is complete, compaction will be retried again, which will eventually hit this code block where the respective commit is
// already part of completed commit. So, we have to manually remove the completed instant and proceed.
// and it is for the same reason we enabled withAllowMultiWriteOnSameInstant for metadata table.
HoodieInstant alreadyCompletedInstant = metadataMetaClient.getActiveTimeline().filterCompletedInstants().filter(entry -> entry.getTimestamp().equals(instantTime)).lastInstant().get();
HoodieActiveTimeline.deleteInstantFile(metadataMetaClient.getFs(), metadataMetaClient.getMetaPath(), alreadyCompletedInstant);
metadataMetaClient.reloadActiveTimeline();
}
List<WriteStatus> statuses = writeClient.upsertPreppedRecords(recordRDD, instantTime).collect();
statuses.forEach(writeStatus -> {
if (writeStatus.hasErrors()) {
throw new HoodieMetadataException("Failed to commit metadata table records at instant " + instantTime);
}
});
// trigger cleaning, compaction, with suffixes based on the same instant time. This ensures that any future
// delta commits synced over will not have an instant time lesser than the last completed instant on the
// metadata table.
if (writeClient.scheduleCompactionAtInstant(instantTime + "001", Option.empty())) {
writeClient.compact(instantTime + "001");
}
writeClient.clean(instantTime + "002");
// reload timeline
metadataMetaClient.reloadActiveTimeline();
compactIfNecessary(writeClient, instantTime);
doClean(writeClient, instantTime);
}
// Update total size of the metadata and count of base/log files
metrics.ifPresent(m -> {
try {
Map<String, String> stats = m.getStats(false, metaClient, metadata);
m.updateMetrics(Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_BASE_FILE_SIZE)),
Long.parseLong(stats.get(HoodieMetadataMetrics.STAT_TOTAL_LOG_FILE_SIZE)),
Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_BASE_FILES)),
Integer.parseInt(stats.get(HoodieMetadataMetrics.STAT_COUNT_LOG_FILES)));
} catch (HoodieIOException e) {
LOG.error("Could not publish metadata size metrics", e);
}
});
metrics.ifPresent(m -> m.updateSizeMetrics(metadataMetaClient, metadata));
}
/**
* Tag each record with the location.
* Perform a compaction on the Metadata Table.
*
* Since we only read the latest base file in a partition, we tag the records with the instant time of the latest
* base file.
* Cases to be handled:
* 1. We cannot perform compaction if there are previous inflight operations on the dataset. This is because
* a compacted metadata base file at time Tx should represent all the actions on the dataset till time Tx.
*
* 2. In multi-writer scenario, a parallel operation with a greater instantTime may have completed creating a
* deltacommit.
*/
private JavaRDD<HoodieRecord> prepRecords(List<HoodieRecord> records, String partitionName) {
HoodieTable table = HoodieSparkTable.create(metadataWriteConfig, engineContext);
TableFileSystemView.SliceView fsView = table.getSliceView();
List<HoodieBaseFile> baseFiles = fsView.getLatestFileSlices(partitionName)
.map(FileSlice::getBaseFile)
.filter(Option::isPresent)
.map(Option::get)
.collect(Collectors.toList());
private void compactIfNecessary(SparkRDDWriteClient writeClient, String instantTime) {
String latestDeltacommitTime = metadataMetaClient.getActiveTimeline().getDeltaCommitTimeline().filterCompletedInstants().lastInstant()
.get().getTimestamp();
List<HoodieInstant> pendingInstants = dataMetaClient.reloadActiveTimeline().filterInflightsAndRequested()
.findInstantsBefore(latestDeltacommitTime).getInstants().collect(Collectors.toList());
// All the metadata fits within a single base file
if (partitionName.equals(MetadataPartitionType.FILES.partitionPath())) {
if (baseFiles.size() > 1) {
throw new HoodieMetadataException("Multiple base files found in metadata partition");
}
if (!pendingInstants.isEmpty()) {
LOG.info(String.format("Cannot compact metadata table as there are %d inflight instants before latest deltacommit %s: %s",
pendingInstants.size(), latestDeltacommitTime, Arrays.toString(pendingInstants.toArray())));
return;
}
// Trigger compaction with suffixes based on the same instant time. This ensures that any future
// delta commits synced over will not have an instant time lesser than the last completed instant on the
// metadata table.
final String compactionInstantTime = latestDeltacommitTime + "001";
if (writeClient.scheduleCompactionAtInstant(compactionInstantTime, Option.empty())) {
writeClient.compact(compactionInstantTime);
}
}
private void doClean(SparkRDDWriteClient writeClient, String instantTime) {
// Trigger cleaning with suffixes based on the same instant time. This ensures that any future
// delta commits synced over will not have an instant time lesser than the last completed instant on the
// metadata table.
writeClient.clean(instantTime + "002");
}
/**
* Tag each record with the location in the given partition.
*
* The record is tagged with respective file slice's location based on its record key.
*/
private JavaRDD<HoodieRecord> prepRecords(List<HoodieRecord> records, String partitionName, int numFileGroups) {
List<FileSlice> fileSlices = HoodieTableMetadataUtil.loadPartitionFileGroupsWithLatestFileSlices(metadataMetaClient, partitionName);
ValidationUtils.checkArgument(fileSlices.size() == numFileGroups, String.format("Invalid number of file groups: found=%d, required=%d", fileSlices.size(), numFileGroups));
JavaSparkContext jsc = ((HoodieSparkEngineContext) engineContext).getJavaSparkContext();
String fileId;
String instantTime;
if (!baseFiles.isEmpty()) {
fileId = baseFiles.get(0).getFileId();
instantTime = baseFiles.get(0).getCommitTime();
} else {
// If there is a log file then we can assume that it has the data
List<HoodieLogFile> logFiles = fsView.getLatestFileSlices(MetadataPartitionType.FILES.partitionPath())
.map(FileSlice::getLatestLogFile)
.filter(Option::isPresent)
.map(Option::get)
.collect(Collectors.toList());
if (logFiles.isEmpty()) {
// No base and log files. All are new inserts
return jsc.parallelize(records, 1);
}
fileId = logFiles.get(0).getFileId();
instantTime = logFiles.get(0).getBaseCommitTime();
}
return jsc.parallelize(records, 1).map(r -> r.setCurrentLocation(new HoodieRecordLocation(instantTime, fileId)));
return jsc.parallelize(records, 1).map(r -> {
FileSlice slice = fileSlices.get(HoodieTableMetadataUtil.mapRecordKeyToFileGroupIndex(r.getRecordKey(), numFileGroups));
r.setCurrentLocation(new HoodieRecordLocation(slice.getBaseInstantTime(), slice.getFileId()));
return r;
});
}
}

View File

@@ -29,14 +29,24 @@ import org.apache.hudi.common.table.timeline.versioning.TimelineLayoutVersion;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.index.SparkHoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import java.io.IOException;
public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
extends HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
private boolean isMetadataAvailabilityUpdated = false;
private boolean isMetadataTableAvailable;
protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
super(config, context, metaClient);
}
@@ -85,4 +95,31 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
protected HoodieIndex<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getIndex(HoodieWriteConfig config, HoodieEngineContext context) {
return SparkHoodieIndex.createIndex(config);
}
/**
* Fetch instance of {@link HoodieTableMetadataWriter}.
*
* @return instance of {@link HoodieTableMetadataWriter}
*/
@Override
public Option<HoodieTableMetadataWriter> getMetadataWriter() {
synchronized (this) {
if (!isMetadataAvailabilityUpdated) {
// this code assumes that if metadata availability is updated once it will not change. please revisit this logic if that's not the case.
// this is done to avoid repeated calls to fs.exists().
try {
isMetadataTableAvailable = config.isMetadataTableEnabled()
&& metaClient.getFs().exists(new Path(HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())));
} catch (IOException e) {
throw new HoodieMetadataException("Checking existence of metadata table failed", e);
}
isMetadataAvailabilityUpdated = true;
}
}
if (isMetadataTableAvailable) {
return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context));
} else {
return Option.empty();
}
}
}

View File

@@ -59,13 +59,10 @@ import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieKeyGeneratorException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.execution.SparkBoundedInMemoryExecutor;
import org.apache.hudi.io.HoodieBootstrapHandle;
import org.apache.hudi.keygen.KeyGeneratorInterface;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -226,17 +223,6 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
LOG.info("Committing metadata bootstrap !!");
}
@Override
protected void syncTableMetadata() {
// Open up the metadata table again, for syncing
try (HoodieTableMetadataWriter writer =
SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
LOG.info("Successfully synced to metadata table");
} catch (Exception e) {
throw new HoodieMetadataException("Error syncing to metadata table.", e);
}
}
protected void commit(Option<Map<String, String>> extraMetadata, HoodieWriteMetadata<JavaRDD<WriteStatus>> result, List<HoodieWriteStat> stats) {
String actionType = table.getMetaClient().getCommitActionType();
LOG.info("Committing " + instantTime + ", action Type " + actionType);
@@ -252,7 +238,6 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
// Finalize write
finalizeWrite(instantTime, stats, result);
syncTableMetadata();
// add in extra metadata
if (extraMetadata.isPresent()) {
extraMetadata.get().forEach(metadata::addMetadata);
@@ -260,6 +245,8 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
metadata.setOperationType(operationType);
writeTableMetadata(metadata);
try {
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));

View File

@@ -40,7 +40,6 @@ import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieIOException;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.execution.SparkLazyInsertIterable;
import org.apache.hudi.io.CreateHandleFactory;
@@ -49,8 +48,6 @@ import org.apache.hudi.io.HoodieSortedMergeHandle;
import org.apache.hudi.io.storage.HoodieConcatHandle;
import org.apache.hudi.keygen.BaseKeyGenerator;
import org.apache.hudi.keygen.factory.HoodieSparkKeyGeneratorFactory;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.WorkloadProfile;
@@ -239,7 +236,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
result.setWriteStatuses(statuses);
return statuses;
}
protected void updateIndexAndCommitIfNeeded(JavaRDD<WriteStatus> writeStatusRDD, HoodieWriteMetadata result) {
updateIndex(writeStatusRDD, result);
result.setPartitionToReplaceFileIds(getPartitionToReplacedFileIds(result));
@@ -264,13 +261,11 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
result.setWriteStats(writeStats);
// Finalize write
finalizeWrite(instantTime, writeStats, result);
syncTableMetadata();
try {
LOG.info("Committing " + instantTime + ", action Type " + getCommitActionType());
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
writeTableMetadata(metadata);
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
LOG.info("Committed " + instantTime);
@@ -354,17 +349,6 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
}
}
@Override
public void syncTableMetadata() {
// Open up the metadata table again, for syncing
try (HoodieTableMetadataWriter writer =
SparkHoodieBackedTableMetadataWriter.create(hadoopConf, config, context)) {
LOG.info("Successfully synced to metadata table");
} catch (Exception e) {
throw new HoodieMetadataException("Error syncing to metadata table.", e);
}
}
@Override
public Iterator<List<WriteStatus>> handleInsert(String idPfx, Iterator<HoodieRecord<T>> recordItr)
throws Exception {

View File

@@ -30,4 +30,4 @@ public class OneToTwoUpgradeHandler extends BaseOneToTwoUpgradeHandler {
String getPartitionColumns(HoodieWriteConfig config) {
return HoodieSparkUtils.getPartitionColumns(config.getProps());
}
}
}

View File

@@ -53,6 +53,8 @@ public class SparkUpgradeDowngrade extends AbstractUpgradeDowngrade {
return new ZeroToOneUpgradeHandler().upgrade(config, context, instantTime);
} else if (fromVersion == HoodieTableVersion.ONE && toVersion == HoodieTableVersion.TWO) {
return new OneToTwoUpgradeHandler().upgrade(config, context, instantTime);
} else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.THREE) {
return new TwoToThreeUpgradeHandler().upgrade(config, context, instantTime);
} else {
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), true);
}
@@ -64,6 +66,8 @@ public class SparkUpgradeDowngrade extends AbstractUpgradeDowngrade {
return new OneToZeroDowngradeHandler().downgrade(config, context, instantTime);
} else if (fromVersion == HoodieTableVersion.TWO && toVersion == HoodieTableVersion.ONE) {
return new TwoToOneDowngradeHandler().downgrade(config, context, instantTime);
} else if (fromVersion == HoodieTableVersion.THREE && toVersion == HoodieTableVersion.TWO) {
return new ThreeToTwoDowngradeHandler().downgrade(config, context, instantTime);
} else {
throw new HoodieUpgradeDowngradeException(fromVersion.versionCode(), toVersion.versionCode(), false);
}

View File

@@ -0,0 +1,44 @@
/*
* 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.upgrade;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.metadata.HoodieTableMetadataUtil;
import java.util.Collections;
import java.util.Map;
/**
* Downgrade handler to assist in downgrading hoodie table from version 3 to 2.
*/
public class ThreeToTwoDowngradeHandler implements DowngradeHandler {
@Override
public Map<ConfigProperty, String> downgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
if (config.isMetadataTableEnabled()) {
// Metadata Table in version 3 is synchronous and in version 2 is asynchronous. Downgrading to asynchronous
// removes the checks in code to decide whether to use a LogBlock or not. Also, the schema for the
// table has been updated and is not forward compatible. Hence, we need to delete the table.
HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context);
}
return Collections.emptyMap();
}
}

View File

@@ -0,0 +1,43 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.upgrade;
import org.apache.hudi.common.config.ConfigProperty;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.metadata.HoodieTableMetadataUtil;
import java.util.Collections;
import java.util.Map;
/**
* UpgradeHandler to assist in upgrading {@link org.apache.hudi.table.HoodieTable} from version 2 to 3.
*/
public class TwoToThreeUpgradeHandler implements UpgradeHandler {
@Override
public Map<ConfigProperty, String> upgrade(HoodieWriteConfig config, HoodieEngineContext context, String instantTime) {
if (config.isMetadataTableEnabled()) {
// Metadata Table in version 2 is asynchronous and in version 3 is synchronous. Synchronous table will not
// sync any instants not already synced. So its simpler to re-bootstrap the table. Also, the schema for the
// table has been updated and is not backward compatible.
HoodieTableMetadataUtil.deleteMetadataTable(config.getBasePath(), context);
}
return Collections.emptyMap();
}
}

View File

@@ -23,6 +23,7 @@ import java.util.Set;
import java.util.stream.Collectors;
import org.apache.hadoop.fs.Path;
import org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass;
import org.apache.hudi.common.config.LockConfiguration;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieTableType;
@@ -42,6 +43,8 @@ import org.apache.hudi.testutils.HoodieClientTestBase;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Disabled;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
@@ -123,15 +126,27 @@ public class TestHoodieClientMultiWriter extends HoodieClientTestBase {
}
}
@ParameterizedTest
@EnumSource(value = HoodieTableType.class, names = {"COPY_ON_WRITE", "MERGE_ON_READ"})
public void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception {
@Disabled
public void testMultiWriterWithAsyncTableServicesWithConflictCOW() throws Exception {
testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.COPY_ON_WRITE);
}
@Test
public void testMultiWriterWithAsyncTableServicesWithConflictMOR() throws Exception {
testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType.MERGE_ON_READ);
}
private void testMultiWriterWithAsyncTableServicesWithConflict(HoodieTableType tableType) throws Exception {
// create inserts X 1
if (tableType == HoodieTableType.MERGE_ON_READ) {
setUpMORTestTable();
}
Properties properties = new Properties();
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath + "/.hoodie/.locks");
properties.setProperty("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.FileSystemBasedLockProviderTestClass");
properties.setProperty(FILESYSTEM_LOCK_PATH_PROP_KEY, basePath);
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_NUM_RETRIES_PROP_KEY, "3");
properties.setProperty(LockConfiguration.LOCK_ACQUIRE_CLIENT_RETRY_WAIT_TIME_IN_MILLIS_PROP_KEY, "5000");
// Disabling embedded timeline server, it doesn't work with multiwriter
HoodieWriteConfig cfg = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withAutoClean(false)

View File

@@ -18,6 +18,7 @@
package org.apache.hudi.client;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.HoodieAvroPayload;
import org.apache.hudi.common.model.HoodieRecord;
@@ -30,6 +31,7 @@ import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.constant.KeyGeneratorOptions;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hudi.testutils.HoodieClientTestUtils;
@@ -67,9 +69,15 @@ public class TestMultiFS extends HoodieClientTestHarness {
}
protected HoodieWriteConfig getHoodieWriteConfig(String basePath) {
return getHoodieWriteConfig(basePath, HoodieMetadataConfig.ENABLE.defaultValue());
}
protected HoodieWriteConfig getHoodieWriteConfig(String basePath, boolean enableMetadata) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withEmbeddedTimelineServerEnabled(true)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2).forTable(tableName)
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build()).build();
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build())
.build();
}
@Test
@@ -82,8 +90,17 @@ public class TestMultiFS extends HoodieClientTestHarness {
.initTable(hadoopConf, dfsBasePath);
// Create write client to write some records in
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath);
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath);
HoodieWriteConfig cfg = getHoodieWriteConfig(dfsBasePath, false);
HoodieWriteConfig localConfig = getHoodieWriteConfig(tablePath, false);
HoodieTableMetaClient.withPropertyBuilder()
.setTableType(tableType)
.setTableName(tableName)
.setPayloadClass(HoodieAvroPayload.class)
.setRecordKeyFields(localConfig.getProps().getProperty(KeyGeneratorOptions.RECORDKEY_FIELD_NAME.key()))
.setPartitionFields(localConfig.getProps().getProperty(KeyGeneratorOptions.PARTITIONPATH_FIELD_NAME.key()))
.initTable(hadoopConf, tablePath);
try (SparkRDDWriteClient hdfsWriteClient = getHoodieWriteClient(cfg);
SparkRDDWriteClient localWriteClient = getHoodieWriteClient(localConfig)) {

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.client.functional;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
@@ -335,7 +336,8 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
@Test
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
// Load to memory
HoodieWriteConfig config = getConfig();
HoodieWriteConfig config = getConfigBuilder(100, false, false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
@@ -422,7 +424,8 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
@Test
public void testEnsureTagLocationUsesCommitTimeline() throws Exception {
// Load to memory
HoodieWriteConfig config = getConfig();
HoodieWriteConfig config = getConfigBuilder(100, false, false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);

View File

@@ -0,0 +1,93 @@
/*
* 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.functional;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.table.view.TableFileSystemView;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.metadata.HoodieBackedTableMetadata;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hadoop.fs.FileStatus;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collections;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertTrue;
import static org.junit.jupiter.api.Assertions.fail;
public class TestHoodieBackedTableMetadata extends TestHoodieMetadataBase {
private static final Logger LOG = LogManager.getLogger(TestHoodieBackedTableMetadata.class);
@Test
public void testTableOperations() throws Exception {
HoodieTableType tableType = HoodieTableType.COPY_ON_WRITE;
init(tableType);
doWriteInsertAndUpsert(testTable);
// trigger an upsert
doWriteOperation(testTable, "0000003");
verifyBaseMetadataTable();
}
private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception {
doWriteInsertAndUpsert(testTable, "0000001", "0000002");
}
private void verifyBaseMetadataTable() throws IOException {
HoodieBackedTableMetadata tableMetadata = new HoodieBackedTableMetadata(context, writeConfig.getMetadataConfig(), writeConfig.getBasePath(), writeConfig.getSpillableMapBasePath(), false);
assertTrue(tableMetadata.enabled());
List<java.nio.file.Path> fsPartitionPaths = testTable.getAllPartitionPaths();
List<String> fsPartitions = new ArrayList<>();
fsPartitionPaths.forEach(entry -> fsPartitions.add(entry.getFileName().toString()));
List<String> metadataPartitions = tableMetadata.getAllPartitionPaths();
Collections.sort(fsPartitions);
Collections.sort(metadataPartitions);
assertEquals(fsPartitions.size(), metadataPartitions.size(), "Partitions should match");
assertEquals(fsPartitions, metadataPartitions, "Partitions should match");
// Files within each partition should match
HoodieTable table = HoodieSparkTable.create(writeConfig, context, true);
TableFileSystemView tableView = table.getHoodieView();
List<String> fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList());
Map<String, FileStatus[]> partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths);
assertEquals(fsPartitions.size(), partitionToFilesMap.size());
fsPartitions.forEach(partition -> {
try {
validateFilesPerPartition(testTable, tableMetadata, tableView, partitionToFilesMap, partition);
} catch (IOException e) {
fail("Exception should not be raised: " + e);
}
});
}
}

View File

@@ -2147,19 +2147,18 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
assertTrue(timeline.getCommitsTimeline().filterCompletedInstants().countInstants() == 0);
}
@ParameterizedTest
@MethodSource("populateMetaFieldsParams")
public void testParallelInsertAndCleanPreviousFailedCommits(boolean populateMetaFields) throws Exception {
@Test
public void testParallelInsertAndCleanPreviousFailedCommits() throws Exception {
HoodieFailedWritesCleaningPolicy cleaningPolicy = HoodieFailedWritesCleaningPolicy.LAZY;
ExecutorService service = Executors.newFixedThreadPool(2);
HoodieTestUtils.init(hadoopConf, basePath);
// Perform 2 failed writes to table
SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
SparkRDDWriteClient client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
writeBatch(client, "100", "100", Option.of(Arrays.asList("100")), "100",
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100,
0, false);
client.close();
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
writeBatch(client, "200", "200", Option.of(Arrays.asList("200")), "200",
100, dataGen::generateInserts, SparkRDDWriteClient::bulkInsert, false, 100, 100,
0, false);
@@ -2167,7 +2166,7 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
// refresh data generator to delete records generated from failed commits
dataGen = new HoodieTestDataGenerator();
// Create a succesful commit
Future<JavaRDD<WriteStatus>> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)),
Future<JavaRDD<WriteStatus>> commit3 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)),
"300", "200", Option.of(Arrays.asList("300")), "200", 100, dataGen::generateInserts,
SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true));
commit3.get();
@@ -2177,17 +2176,17 @@ public class TestHoodieClientOnCopyOnWriteStorage extends HoodieClientTestBase {
CollectionUtils.createSet(HoodieTimeline.ROLLBACK_ACTION)).countInstants() == 0);
assertTrue(metaClient.getActiveTimeline().filterInflights().countInstants() == 2);
assertTrue(metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().countInstants() == 1);
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields));
client = new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true));
// Await till enough time passes such that the first 2 failed commits heartbeats are expired
boolean conditionMet = false;
while (!conditionMet) {
conditionMet = client.getHeartbeatClient().isHeartbeatExpired("200");
Thread.sleep(2000);
}
Future<JavaRDD<WriteStatus>> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)),
Future<JavaRDD<WriteStatus>> commit4 = service.submit(() -> writeBatch(new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)),
"400", "300", Option.of(Arrays.asList("400")), "400", 100, dataGen::generateInserts,
SparkRDDWriteClient::bulkInsert, false, 100, 100, 0, true));
Future<HoodieCleanMetadata> clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, populateMetaFields)).clean());
Future<HoodieCleanMetadata> clean1 = service.submit(() -> new SparkRDDWriteClient(context, getParallelWritingWriteConfig(cleaningPolicy, true)).clean());
commit4.get();
clean1.get();
HoodieActiveTimeline timeline = metaClient.getActiveTimeline().reload();

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.client.functional;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.ConsistencyGuardConfig;
import org.apache.hudi.common.model.EmptyHoodieRecordPayload;
import org.apache.hudi.common.model.HoodieKey;
@@ -97,6 +98,10 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
private HoodieWriteConfig config;
private void setUp(IndexType indexType, boolean populateMetaFields) throws Exception {
setUp(indexType, populateMetaFields, true);
}
private void setUp(IndexType indexType, boolean populateMetaFields, boolean enableMetadata) throws Exception {
this.indexType = indexType;
initPath();
initSparkContexts();
@@ -107,7 +112,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
config = getConfigBuilder()
.withProperties(populateMetaFields ? new Properties() : getPropertiesForKeyGen())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
.build()).withAutoCommit(false).build();
.build()).withAutoCommit(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata).build()).build();
writeClient = getHoodieWriteClient(config);
this.index = writeClient.getIndex();
}
@@ -220,7 +225,7 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
@ParameterizedTest
@MethodSource("indexTypeParams")
public void testSimpleTagLocationAndUpdateWithRollback(IndexType indexType, boolean populateMetaFields) throws Exception {
setUp(indexType, populateMetaFields);
setUp(indexType, populateMetaFields, false);
String newCommitTime = writeClient.startCommit();
int totalRecords = 20 + random.nextInt(20);
List<HoodieRecord> records = dataGen.generateInserts(newCommitTime, totalRecords);
@@ -367,7 +372,8 @@ public class TestHoodieIndex extends HoodieClientTestHarness {
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(indexType)
.withGlobalSimpleIndexUpdatePartitionPath(true)
.withBloomIndexUpdatePartitionPath(true)
.build()).build();
.build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
writeClient = getHoodieWriteClient(config);
index = writeClient.getIndex();
HoodieTable hoodieTable = HoodieSparkTable.create(config, context, metaClient);

View File

@@ -0,0 +1,281 @@
/*
* 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.functional;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.view.FileSystemViewStorageConfig;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieIndexConfig;
import org.apache.hudi.config.HoodieStorageConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.config.metrics.HoodieMetricsConfig;
import org.apache.hudi.config.metrics.HoodieMetricsGraphiteConfig;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.AfterEach;
import java.io.IOException;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.apache.hudi.common.model.WriteOperationType.INSERT;
import static org.apache.hudi.common.model.WriteOperationType.UPSERT;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA;
public class TestHoodieMetadataBase extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieMetadataBase.class);
protected static HoodieTestTable testTable;
protected String metadataTableBasePath;
protected HoodieTableType tableType;
protected HoodieWriteConfig writeConfig;
protected HoodieTableMetadataWriter metadataWriter;
public void init(HoodieTableType tableType) throws IOException {
init(tableType, true);
}
public void init(HoodieTableType tableType, boolean enableMetadataTable) throws IOException {
this.tableType = tableType;
initPath();
initSparkContexts("TestHoodieMetadata");
initFileSystem();
fs.mkdirs(new Path(basePath));
initMetaClient(tableType);
initTestDataGenerator();
metadataTableBasePath = HoodieTableMetadata.getMetadataTableBasePath(basePath);
writeConfig = getWriteConfig(true, enableMetadataTable);
initWriteConfigAndMetatableWriter(writeConfig, enableMetadataTable);
}
protected void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) {
this.writeConfig = writeConfig;
if (enableMetadataTable) {
metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context);
testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter);
} else {
testTable = HoodieTestTable.of(metaClient);
}
}
@AfterEach
public void clean() throws Exception {
cleanupResources();
}
protected void doWriteInsertAndUpsert(HoodieTestTable testTable, String commit1, String commit2) throws Exception {
testTable.doWriteOperation(commit1, INSERT, asList("p1", "p2"), asList("p1", "p2"),
4, false);
testTable.doWriteOperation(commit2, UPSERT, asList("p1", "p2"),
4, false);
validateMetadata(testTable);
}
protected void doWriteOperationAndValidateMetadata(HoodieTestTable testTable, String commitTime) throws Exception {
doWriteOperation(testTable, commitTime);
validateMetadata(testTable);
}
protected void doWriteOperation(HoodieTestTable testTable, String commitTime) throws Exception {
doWriteOperation(testTable, commitTime, UPSERT);
}
protected void doWriteOperationAndValidate(HoodieTestTable testTable, String commitTime) throws Exception {
doWriteOperationAndValidate(testTable, commitTime, UPSERT);
}
protected void doWriteOperationAndValidate(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception {
doWriteOperation(testTable, commitTime, operationType);
validateMetadata(testTable);
}
protected void doWriteOperation(HoodieTestTable testTable, String commitTime, WriteOperationType operationType) throws Exception {
testTable.doWriteOperation(commitTime, operationType, emptyList(), asList("p1", "p2"), 3);
}
protected void doClean(HoodieTestTable testTable, String commitTime, List<String> commitsToClean) throws IOException {
doCleanInternal(testTable, commitTime, commitsToClean, false);
}
protected void doCleanAndValidate(HoodieTestTable testTable, String commitTime, List<String> commitsToClean) throws IOException {
doCleanInternal(testTable, commitTime, commitsToClean, true);
}
private void doCleanInternal(HoodieTestTable testTable, String commitTime, List<String> commitsToClean, boolean validate) throws IOException {
testTable.doCleanBasedOnCommits(commitTime, commitsToClean);
if (validate) {
validateMetadata(testTable);
}
}
protected void doCompaction(HoodieTestTable testTable, String commitTime) throws Exception {
doCompactionInternal(testTable, commitTime, false);
}
protected void doCompactionAndValidate(HoodieTestTable testTable, String commitTime) throws Exception {
doCompactionInternal(testTable, commitTime, true);
}
private void doCompactionInternal(HoodieTestTable testTable, String commitTime, boolean validate) throws Exception {
testTable.doCompaction(commitTime, asList("p1", "p2"));
if (validate) {
validateMetadata(testTable);
}
}
protected void doCluster(HoodieTestTable testTable, String commitTime) throws Exception {
doClusterInternal(testTable, commitTime, false);
}
protected void doClusterAndValidate(HoodieTestTable testTable, String commitTime) throws Exception {
doClusterInternal(testTable, commitTime, true);
}
protected void doClusterInternal(HoodieTestTable testTable, String commitTime, boolean validate) throws Exception {
testTable.doCluster(commitTime, new HashMap<>(), Arrays.asList("p1", "p2"), 2);
if (validate) {
validateMetadata(testTable);
}
}
protected void doRollback(HoodieTestTable testTable, String commitToRollback, String rollbackTime) throws Exception {
doRollbackInternal(testTable, commitToRollback, rollbackTime, false);
}
protected void doRollbackAndValidate(HoodieTestTable testTable, String commitToRollback, String rollbackTime) throws Exception {
doRollbackInternal(testTable, commitToRollback, rollbackTime, true);
}
private void doRollbackInternal(HoodieTestTable testTable, String commitToRollback, String rollbackTime, boolean validate) throws Exception {
testTable.doRollback(commitToRollback, rollbackTime);
if (validate) {
validateMetadata(testTable);
}
}
protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, String commitTime) throws Exception {
doPreBootstrapWriteOperation(testTable, UPSERT, commitTime);
}
protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, WriteOperationType writeOperationType, String commitTime) throws Exception {
doPreBootstrapWriteOperation(testTable, writeOperationType, commitTime, 2);
}
protected void doPreBootstrapWriteOperation(HoodieTestTable testTable, WriteOperationType writeOperationType, String commitTime, int filesPerPartition) throws Exception {
testTable.doWriteOperation(commitTime, writeOperationType, asList("p1", "p2"), asList("p1", "p2"),
filesPerPartition, true);
}
protected void doPreBootstrapClean(HoodieTestTable testTable, String commitTime, List<String> commitsToClean) throws Exception {
testTable.doCleanBasedOnCommits(commitTime, commitsToClean);
}
protected void doPreBootstrapRollback(HoodieTestTable testTable, String rollbackTime, String commitToRollback) throws Exception {
testTable.doRollback(commitToRollback, rollbackTime);
}
protected void doPrebootstrapCompaction(HoodieTestTable testTable, String commitTime) throws Exception {
doPrebootstrapCompaction(testTable, commitTime, Arrays.asList("p1", "p2"));
}
protected void doPrebootstrapCompaction(HoodieTestTable testTable, String commitTime, List<String> partitions) throws Exception {
testTable.doCompaction(commitTime, partitions);
}
protected void doPreBootstrapCluster(HoodieTestTable testTable, String commitTime) throws Exception {
testTable.doCluster(commitTime, new HashMap<>(), Arrays.asList("p1", "p2"), 2);
}
protected void doPreBootstrapRestore(HoodieTestTable testTable, String restoreTime, String commitToRestore) throws Exception {
testTable.doRestore(commitToRestore, restoreTime);
}
protected void archiveDataTable(HoodieWriteConfig writeConfig, HoodieTableMetaClient metaClient) throws IOException {
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
archiveLog.archiveIfRequired(context);
}
protected void validateMetadata(HoodieTestTable testTable) throws IOException {
validateMetadata(testTable, emptyList());
}
protected void validateMetadata(HoodieTestTable testTable, boolean doFullValidation) throws IOException {
validateMetadata(testTable, emptyList(), doFullValidation);
}
protected void validateMetadata(HoodieTestTable testTable, List<String> inflightCommits) throws IOException {
validateMetadata(testTable, inflightCommits, false);
}
protected void validateMetadata(HoodieTestTable testTable, List<String> inflightCommits, boolean doFullValidation) throws IOException {
validateMetadata(testTable, inflightCommits, writeConfig, metadataTableBasePath, doFullValidation);
}
protected HoodieWriteConfig getWriteConfig(boolean autoCommit, boolean useFileListingMetadata) {
return getWriteConfigBuilder(autoCommit, useFileListingMetadata, false).build();
}
protected HoodieWriteConfig.Builder getWriteConfigBuilder(boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) {
return getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy.EAGER, autoCommit, useFileListingMetadata, enableMetrics);
}
protected HoodieWriteConfig.Builder getWriteConfigBuilder(HoodieFailedWritesCleaningPolicy policy, boolean autoCommit, boolean useFileListingMetadata, boolean enableMetrics) {
return HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).withDeleteParallelism(2).withRollbackParallelism(2).withFinalizeWriteParallelism(2)
.withAutoCommit(autoCommit)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().compactionSmallFileSize(1024 * 1024 * 1024)
.withInlineCompaction(false).withMaxNumDeltaCommitsBeforeCompaction(1)
.withFailedWritesCleaningPolicy(policy)
.withAutoClean(false).retainCommits(1).retainFileVersions(1).build())
.withStorageConfig(HoodieStorageConfig.newBuilder().hfileMaxFileSize(1024 * 1024 * 1024).build())
.withEmbeddedTimelineServerEnabled(true).forTable("test-trip-table")
.withFileSystemViewConfig(new FileSystemViewStorageConfig.Builder()
.withEnableBackupForRemoteFileSystemView(false).build())
.withIndexConfig(HoodieIndexConfig.newBuilder().withIndexType(HoodieIndex.IndexType.BLOOM).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(useFileListingMetadata)
.enableMetrics(enableMetrics).build())
.withMetricsConfig(HoodieMetricsConfig.newBuilder().on(enableMetrics)
.withExecutorMetrics(true).build())
.withMetricsGraphiteConfig(HoodieMetricsGraphiteConfig.newBuilder()
.usePrefix("unit-test").build());
}
}

View File

@@ -0,0 +1,252 @@
/*
* 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.functional;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.EnumSource;
import java.util.Arrays;
import static java.util.Arrays.asList;
import static java.util.Collections.emptyList;
import static org.apache.hudi.common.model.HoodieTableType.COPY_ON_WRITE;
import static org.apache.hudi.common.model.HoodieTableType.MERGE_ON_READ;
import static org.apache.hudi.common.model.WriteOperationType.INSERT;
import static org.apache.hudi.common.model.WriteOperationType.UPSERT;
@Tag("functional")
public class TestHoodieMetadataBootstrap extends TestHoodieMetadataBase {
private static final Logger LOG = LogManager.getLogger(TestHoodieMetadataBootstrap.class);
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapInsertUpsert(HoodieTableType tableType) throws Exception {
init(tableType, false);
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000003");
}
bootstrapAndVerify();
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapInsertUpsertClean(HoodieTableType tableType) throws Exception {
init(tableType, false);
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
doPreBootstrapClean(testTable, "0000003", Arrays.asList("0000001"));
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000004");
}
doPreBootstrapWriteOperation(testTable, "0000005");
bootstrapAndVerify();
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapInsertUpsertRollback(HoodieTableType tableType) throws Exception {
init(tableType, false);
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
doPreBootstrapRollback(testTable, "0000003", "0000002");
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000004");
}
bootstrapAndVerify();
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapInsertUpsertCluster(HoodieTableType tableType) throws Exception {
init(tableType, false);
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
doPreBootstrapCluster(testTable, "0000003");
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000004");
}
bootstrapAndVerify();
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapLargeCommitList(HoodieTableType tableType) throws Exception {
init(tableType, false);
for (int i = 1; i < 25; i += 7) {
String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i;
String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1);
String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2);
String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3);
String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4);
String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5);
String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6);
doPreBootstrapWriteOperation(testTable, INSERT, commitTime1);
doPreBootstrapWriteOperation(testTable, commitTime2);
doPreBootstrapClean(testTable, commitTime3, Arrays.asList(commitTime1));
doPreBootstrapWriteOperation(testTable, commitTime4);
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, commitTime5);
}
doPreBootstrapWriteOperation(testTable, commitTime6);
doPreBootstrapRollback(testTable, commitTime7, commitTime6);
}
bootstrapAndVerify();
}
@Test
public void testMetadataBootstrapInflightCommit() throws Exception {
HoodieTableType tableType = COPY_ON_WRITE;
init(tableType, false);
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
// add an inflight commit
HoodieCommitMetadata inflightCommitMeta = testTable.doWriteOperation("00000007", UPSERT, emptyList(),
asList("p1", "p2"), 2, true, true);
// bootstrap and following validation should fail. bootstrap should not happen.
bootstrapAndVerifyFailure();
// once the commit is complete, metadata should get fully synced.
// in prod code path, SparkHoodieBackedTableMetadataWriter.create() will be called for every commit,
// which may not be the case here if we directly call HoodieBackedTableMetadataWriter.update()
// hence lets first move the commit to complete and invoke sync directly
((HoodieMetadataTestTable) testTable).moveInflightCommitToComplete("00000007", inflightCommitMeta, true);
syncTableMetadata(writeConfig);
validateMetadata(testTable);
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapArchival(HoodieTableType tableType) throws Exception {
init(tableType, false);
writeConfig = getWriteConfig(2, 4);
for (int i = 1; i < 13; i += 7) {
String commitTime1 = ((i > 9) ? ("00000") : ("000000")) + i;
String commitTime2 = ((i > 9) ? ("00000") : ("000000")) + (i + 1);
String commitTime3 = ((i > 9) ? ("00000") : ("000000")) + (i + 2);
String commitTime4 = ((i > 9) ? ("00000") : ("000000")) + (i + 3);
String commitTime5 = ((i > 9) ? ("00000") : ("000000")) + (i + 4);
String commitTime6 = ((i > 9) ? ("00000") : ("000000")) + (i + 5);
String commitTime7 = ((i > 9) ? ("00000") : ("000000")) + (i + 6);
doPreBootstrapWriteOperation(testTable, INSERT, commitTime1);
doPreBootstrapWriteOperation(testTable, commitTime2);
doPreBootstrapClean(testTable, commitTime3, Arrays.asList(commitTime1));
doPreBootstrapWriteOperation(testTable, commitTime4);
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, commitTime5);
}
doPreBootstrapWriteOperation(testTable, commitTime6);
doPreBootstrapRollback(testTable, commitTime7, commitTime6);
}
// archive and then bootstrap
archiveDataTable(writeConfig, metaClient);
bootstrapAndVerify();
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapAfterRestore(HoodieTableType tableType) throws Exception {
init(tableType, false);
testRestore(false);
}
@ParameterizedTest
@EnumSource(HoodieTableType.class)
public void testMetadataBootstrapAfterRestoreAndUpserts(HoodieTableType tableType) throws Exception {
init(tableType, false);
testRestore(true);
}
private void testRestore(boolean addUpsertsAfterRestore) throws Exception {
doPreBootstrapWriteOperation(testTable, INSERT, "0000001");
doPreBootstrapWriteOperation(testTable, "0000002");
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000003");
}
doPreBootstrapWriteOperation(testTable, "0000004");
doPreBootstrapWriteOperation(testTable, "0000005");
doPreBootstrapWriteOperation(testTable, "0000006");
doPreBootstrapRestore(testTable, "0000007", "0000004");
if (addUpsertsAfterRestore) {
doPreBootstrapWriteOperation(testTable, "0000008");
doPreBootstrapWriteOperation(testTable, "0000009");
if (tableType == MERGE_ON_READ) {
doPrebootstrapCompaction(testTable, "0000010");
}
}
bootstrapAndVerify();
}
private void bootstrapAndVerify() throws Exception {
writeConfig = getWriteConfig(true, true);
initWriteConfigAndMetatableWriter(writeConfig, true);
syncTableMetadata(writeConfig);
validateMetadata(testTable);
// after bootstrap do two writes and validate its still functional.
doWriteInsertAndUpsert(testTable);
validateMetadata(testTable);
}
private void bootstrapAndVerifyFailure() throws Exception {
writeConfig = getWriteConfig(true, true);
initWriteConfigAndMetatableWriter(writeConfig, true);
syncTableMetadata(writeConfig);
try {
validateMetadata(testTable);
Assertions.fail("Should have failed");
} catch (IllegalStateException e) {
// expected
}
}
private void doWriteInsertAndUpsert(HoodieTestTable testTable) throws Exception {
doWriteInsertAndUpsert(testTable, "0000100", "0000101");
}
private HoodieWriteConfig getWriteConfig(int minArchivalCommits, int maxArchivalCommits) throws Exception {
return HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.forTable("test-trip-table").build();
}
@Override
protected HoodieTableType getTableType() {
return tableType;
}
}

View File

@@ -24,23 +24,28 @@ import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieRollbackMetadata;
import org.apache.hudi.client.utils.MetadataConversionUtils;
import org.apache.hudi.common.HoodieCleanStat;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.model.HoodieCleaningPolicy;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieTableType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieActiveTimeline;
import org.apache.hudi.common.table.timeline.HoodieArchivedTimeline;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieInstant.State;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.table.timeline.versioning.clean.CleanPlanV2MigrationHandler;
import org.apache.hudi.common.testutils.HoodieMetadataTestTable;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
import org.apache.hudi.common.testutils.HoodieTestTable;
import org.apache.hudi.common.testutils.HoodieTestUtils;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.collection.Pair;
import org.apache.hudi.config.HoodieCompactionConfig;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.HoodieTimelineArchiveLog;
@@ -48,19 +53,23 @@ import org.apache.hudi.testutils.HoodieClientTestHarness;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.junit.jupiter.api.AfterEach;
import org.junit.jupiter.api.BeforeEach;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.Comparator;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.Stream;
@@ -71,28 +80,62 @@ import static org.junit.jupiter.api.Assertions.assertTrue;
public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
private static final Logger LOG = LogManager.getLogger(TestHoodieTimelineArchiveLog.class);
private Configuration hadoopConf;
private HoodieWrapperFileSystem wrapperFs;
private HoodieTableMetadataWriter metadataWriter;
private HoodieTestTable testTable;
@BeforeEach
public void init() throws Exception {
init(HoodieTableType.COPY_ON_WRITE);
}
public void init(HoodieTableType tableType) throws Exception {
initPath();
initSparkContexts();
initMetaClient();
hadoopConf = context.getHadoopConf().get();
metaClient.getFs().mkdirs(new Path(basePath));
metaClient = HoodieTestUtils.init(hadoopConf, basePath);
metaClient = HoodieTestUtils.init(hadoopConf, basePath, tableType);
wrapperFs = metaClient.getFs();
hadoopConf.addResource(wrapperFs.getConf());
}
private void initWriteConfigAndMetatableWriter(HoodieWriteConfig writeConfig, boolean enableMetadataTable) {
if (enableMetadataTable) {
metadataWriter = SparkHoodieBackedTableMetadataWriter.create(hadoopConf, writeConfig, context);
testTable = HoodieMetadataTestTable.of(metaClient, metadataWriter);
} else {
testTable = HoodieTestTable.of(metaClient);
}
}
@AfterEach
public void clean() throws IOException {
cleanupResources();
}
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable) throws Exception {
return initTestTableAndGetWriteConfig(enableMetadata, minArchivalCommits, maxArchivalCommits, maxDeltaCommitsMetadataTable, HoodieTableType.COPY_ON_WRITE);
}
private HoodieWriteConfig initTestTableAndGetWriteConfig(boolean enableMetadata, int minArchivalCommits, int maxArchivalCommits, int maxDeltaCommitsMetadataTable,
HoodieTableType tableType) throws Exception {
init(tableType);
HoodieWriteConfig writeConfig = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minArchivalCommits, maxArchivalCommits).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(enableMetadata)
.withMaxNumDeltaCommitsBeforeCompaction(maxDeltaCommitsMetadataTable).build())
.forTable("test-trip-table").build();
initWriteConfigAndMetatableWriter(writeConfig, enableMetadata);
return writeConfig;
}
@Test
public void testArchiveEmptyTable() throws IOException {
public void testArchiveEmptyTable() throws Exception {
init();
HoodieWriteConfig cfg =
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table").build();
@@ -103,230 +146,71 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
assertTrue(result);
}
@Test
public void testArchiveTableWithArchival() throws IOException {
HoodieWriteConfig cfg = HoodieWriteConfig.newBuilder().withPath(basePath)
.withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(2, 4).build())
.forTable("test-trip-table").build();
HoodieTestUtils.init(hadoopConf, basePath);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105"), wrapperFs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testArchiveTableWithArchival(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTimeline timeline = metaClient.getActiveTimeline().getCommitsTimeline().filterCompletedInstants();
// min archival commits is 2 and max archival commits is 4. and so, after 5th commit, 3 commits will be archived.
// 1,2,3,4,5 : after archival -> 4,5
// after 3 more commits, earliest 3 will be archived
// 4,5,6,7,8 : after archival -> 7, 8
// after 9 no-op wrt archival.
for (int i = 1; i < 10; i++) {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
if (i < 5) {
assertEquals(originalCommits, commitsAfterArchival);
} else if (i == 5) {
// archival should have kicked in.
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003")), getActiveCommitInstants(Arrays.asList("00000004", "00000005")), commitsAfterArchival);
} else if (i < 8) {
assertEquals(originalCommits, commitsAfterArchival);
} else if (i == 8) {
// archival should have kicked in.
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006")),
getActiveCommitInstants(Arrays.asList("00000007", "00000008")), commitsAfterArchival);
} else {
assertEquals(originalCommits, commitsAfterArchival);
}
}
}
assertEquals(6, timeline.countInstants(), "Loaded 6 commits and the count should match");
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testNoArchivalUntilMaxArchiveConfigWithExtraInflightCommits(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 5, 2);
createCleanMetadata("100", false);
createCleanMetadata("101", false);
createCleanMetadata("102", false);
createCleanMetadata("103", false);
createCleanMetadata("104", false);
createCleanMetadata("105", false);
createCleanMetadata("106", true);
createCleanMetadata("107", true);
// when max archival commits is set to 5, until 6th commit there should not be any archival.
for (int i = 1; i < 6; i++) {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2);
// archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits, commitsAfterArchival);
}
// reload the timeline and get all the commmits before archive
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
// add couple of inflight. no archival should kick in.
testTable.doWriteOperation("00000006", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2, false, true);
testTable.doWriteOperation("00000007", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 2, false, true);
assertEquals(12, timeline.countInstants(), "Loaded 6 commits and the count should match");
// verify in-flight instants before archive
verifyInflightInstants(metaClient, 2);
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
assertTrue(archiveLog.archiveIfRequired(context));
// reload the timeline and remove the remaining commits
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
originalCommits.removeAll(timeline.getInstants().collect(Collectors.toList()));
// Check compaction instants
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
assertEquals(4, instants.size(), "Should delete all compaction instants < 104");
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")),
"Requested Compaction must be absent for 100");
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")),
"Inflight Compaction must be absent for 100");
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
"Requested Compaction must be absent for 101");
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")),
"Inflight Compaction must be absent for 101");
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")),
"Requested Compaction must be absent for 102");
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")),
"Inflight Compaction must be absent for 102");
assertFalse(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")),
"Requested Compaction must be absent for 103");
assertFalse(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")),
"Inflight Compaction must be absent for 103");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")),
"Requested Compaction must be present for 104");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "104")),
"Inflight Compaction must be present for 104");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "105")),
"Requested Compaction must be present for 105");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "105")),
"Inflight Compaction must be present for 105");
// read the file
HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient);
assertEquals(24, archivedTimeline.countInstants(),
"Total archived records and total read records are the same count");
//make sure the archived commits are the same as the (originalcommits - commitsleft)
Set<String> readCommits =
archivedTimeline.getInstants().map(HoodieInstant::getTimestamp).collect(Collectors.toSet());
assertEquals(originalCommits.stream().map(HoodieInstant::getTimestamp).collect(Collectors.toSet()), readCommits,
"Read commits map should match the originalCommits - commitsLoadedFromArchival");
// verify in-flight instants after archive
verifyInflightInstants(metaClient, 2);
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits, commitsAfterArchival);
}
@Test
public void testArchiveTableWithNoArchival() throws IOException {
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, 5).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
// Requested Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
// Inflight Compaction
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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");
boolean result = archiveLog.archiveIfRequired(context);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals(4, timeline.countInstants(), "Should not archive commits when maxCommitsToKeep is 5");
List<HoodieInstant> instants = metaClient.scanHoodieInstantsFromFileSystem(
new Path(metaClient.getMetaAuxiliaryPath()), HoodieActiveTimeline.VALID_EXTENSIONS_IN_ACTIVE_TIMELINE, false);
assertEquals(8, instants.size(), "Should not delete any aux compaction files when maxCommitsToKeep is 5");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "100")),
"Requested Compaction must be present for 100");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "100")),
"Inflight Compaction must be present for 100");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
"Requested Compaction must be present for 101");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "101")),
"Inflight Compaction must be present for 101");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "102")),
"Requested Compaction must be present for 102");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "102")),
"Inflight Compaction must be present for 102");
assertTrue(instants.contains(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "103")),
"Requested Compaction must be present for 103");
assertTrue(instants.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "103")),
"Inflight Compaction must be present for 103");
}
@Test
public void testArchiveCommitSafety() throws IOException {
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, 5).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
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");
boolean result = archiveLog.archiveIfRequired(context);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertTrue(timeline.containsOrBeforeTimelineStarts("100"), "Archived commits should always be safe");
assertTrue(timeline.containsOrBeforeTimelineStarts("101"), "Archived commits should always be safe");
assertTrue(timeline.containsOrBeforeTimelineStarts("102"), "Archived commits should always be safe");
assertTrue(timeline.containsOrBeforeTimelineStarts("103"), "Archived commits should always be safe");
}
@Test
public void testArchiveCommitSavepointNoHole() throws IOException {
public void testArchiveCommitSavepointNoHole() throws Exception {
init();
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, 5).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
@@ -353,89 +237,97 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
"Archived commits should always be safe");
}
@Test
public void testArchiveRollbacks() throws IOException {
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();
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testArchiveRollbacksTestTable(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 3, 2);
createCommitAndRollbackFile("100", "101", false);
createCommitAndRollbackFile("102", "103", false);
createCommitAndRollbackFile("104", "105", false);
createCommitAndRollbackFile("106", "107", false);
for (int i = 1; i < 9; i += 2) {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
testTable.doRollback("0000000" + i, "0000000" + (i + 1));
HoodieTable table = HoodieSparkTable.create(cfg, context);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
assertTrue(archiveLog.archiveIfRequired(context));
HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getCommitsTimeline().filterCompletedInstants();
assertEquals(2, timeline.countInstants(),
"first two commits must have been archived");
assertFalse(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "101")),
"first rollback must have been archived");
assertFalse(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "103")),
"second rollback must have been archived");
assertTrue(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "105")),
"first rollback must have been archived");
assertTrue(metaClient.getActiveTimeline().containsInstant(new HoodieInstant(false, HoodieTimeline.ROLLBACK_ACTION, "107")),
"second rollback must have been archived");
if (i != 7) {
assertEquals(originalCommits, commitsAfterArchival);
} else {
// only time when archival will kick in
List<HoodieInstant> expectedArchivedInstants = new ArrayList<>();
expectedArchivedInstants.addAll(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003")));
expectedArchivedInstants.addAll(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000004"), HoodieTimeline.ROLLBACK_ACTION));
List<HoodieInstant> expectedActiveInstants = new ArrayList<>();
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000005", "00000007")));
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000006", "00000008"), HoodieTimeline.ROLLBACK_ACTION));
verifyArchival(expectedArchivedInstants, expectedActiveInstants, commitsAfterArchival);
}
}
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testNoArchivalWithInflightCompactionInMiddle(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2,
HoodieTableType.MERGE_ON_READ);
// when max archival commits is set to 4, even after 7 commits, if there is an inflight compaction in the middle, archival should not kick in.
HoodieCommitMetadata inflightCompactionMetadata = null;
for (int i = 1; i < 8; i++) {
if (i == 2) {
inflightCompactionMetadata = testTable.doCompaction("0000000" + i, Arrays.asList("p1", "p2"), true);
} else {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
}
// archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
if (i != 6) {
assertEquals(originalCommits, commitsAfterArchival);
} else {
// on 6th commit, archival will kick in. but will archive only one commit since 2nd compaction commit is inflight.
assertEquals(originalCommits.size() - commitsAfterArchival.size(), 1);
for (int j = 1; j <= 6; j++) {
if (j == 1) {
// first commit should be archived
assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j)));
} else if (j == 2) {
// 2nd compaction should not be archived
assertFalse(commitsAfterArchival.contains(new HoodieInstant(State.INFLIGHT, HoodieTimeline.COMPACTION_ACTION, "0000000" + j)));
} else {
// every other commit should not be archived
assertTrue(commitsAfterArchival.contains(new HoodieInstant(State.COMPLETED, HoodieTimeline.DELTA_COMMIT_ACTION, "0000000" + j)));
}
}
}
}
// move inflight compaction to complete. archival should archive more commits.
// before this move, timeline 2_inflight_compaction, 3,4,5,6,7.
// after this move. 6,7. (2,3,4,5 will be archived)
testTable.moveInflightCompactionToComplete("00000002", inflightCompactionMetadata);
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
List<HoodieInstant> archivedInstants = getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003", "00000004", "00000005"), HoodieTimeline.DELTA_COMMIT_ACTION);
archivedInstants.add(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "00000002"));
archivedInstants.add(new HoodieInstant(State.COMPLETED, HoodieTimeline.COMMIT_ACTION, "00000002"));
verifyArchival(archivedInstants, getActiveCommitInstants(Arrays.asList("00000006", "00000007"), HoodieTimeline.DELTA_COMMIT_ACTION), commitsAfterArchival);
}
@Test
public void testArchiveCommitCompactionNoHole() throws IOException {
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, 5).build())
.build();
HoodieTestDataGenerator.createCommitFile(basePath, "100", wrapperFs.getConf());
HoodieTestDataGenerator.createCompactionRequestedFile(basePath, "101", wrapperFs.getConf());
HoodieTestDataGenerator.createCompactionAuxiliaryMetadata(basePath,
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"), wrapperFs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "105", wrapperFs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "106", wrapperFs.getConf());
HoodieTestDataGenerator.createCommitFile(basePath, "107", wrapperFs.getConf());
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
HoodieTimeline timeline = metaClient.getActiveTimeline().getWriteTimeline();
assertEquals(8, timeline.countInstants(), "Loaded 6 commits and the count should match");
boolean result = archiveLog.archiveIfRequired(context);
assertTrue(result);
timeline = metaClient.getActiveTimeline().reload().getWriteTimeline();
assertFalse(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "100")),
"Instants before oldest pending compaction can be removed");
assertEquals(7, timeline.countInstants(),
"Since we have a pending compaction at 101, we should never archive any commit "
+ "after 101 (we only archive 100)");
assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "101")),
"Requested Compaction must still be present");
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "102")),
"Instants greater than oldest pending compaction must be present");
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "103")),
"Instants greater than oldest pending compaction must be present");
assertTrue(timeline.containsInstant(new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, "104")),
"Instants greater than oldest pending compaction must be present");
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "105")),
"Instants greater than oldest pending compaction must be present");
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "106")),
"Instants greater than oldest pending compaction must be present");
assertTrue(timeline.containsInstant(new HoodieInstant(false, HoodieTimeline.COMMIT_ACTION, "107")),
"Instants greater than oldest pending compaction must be present");
}
@Test
public void testArchiveCommitTimeline() throws IOException {
public void testArchiveCommitTimeline() throws Exception {
init();
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();
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())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTestDataGenerator.createCommitFile(basePath, "1", wrapperFs.getConf());
@@ -470,7 +362,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
}
@Test
public void testConvertCommitMetadata() {
public void testConvertCommitMetadata() throws Exception {
init();
HoodieCommitMetadata hoodieCommitMetadata = new HoodieCommitMetadata();
hoodieCommitMetadata.setOperationType(WriteOperationType.INSERT);
@@ -481,110 +374,93 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
assertEquals(expectedCommitMetadata.getOperationType(), WriteOperationType.INSERT.toString());
}
@Test
public void testArchiveCompletedClean() throws IOException {
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();
metaClient = HoodieTableMetaClient.reload(metaClient);
@ParameterizedTest
@ValueSource(booleans = {true, false})
public void testArchiveTableWithCleanCommits(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2);
createCleanMetadata("10", false);
createCleanMetadata("11", false);
HoodieInstant notArchivedInstant1 = createCleanMetadata("12", false);
HoodieInstant notArchivedInstant2 = createCleanMetadata("13", false);
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
List<HoodieInstant> notArchivedInstants = metaClient.getActiveTimeline().reload().getInstants().collect(Collectors.toList());
//There will be 3 * 2 files but due to TimelineLayoutV1 this will show as 2.
assertEquals(2, notArchivedInstants.size(), "Not archived instants should be 2");
assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2), "");
// min archival commits is 2 and max archival commits is 4(either clean commits has to be > 4 or commits has to be greater than 4. and so, after 5th commit, 3 commits will be archived.
// 1,2,3,4,5,6 : after archival -> 1,5,6 (because, 2,3,4,5 and 6 are clean commits and are eligible for archival)
// after 7th and 8th commit no-op wrt archival.
Map<String, Integer> cleanStats = new HashMap<>();
cleanStats.put("p1", 1);
cleanStats.put("p2", 2);
for (int i = 1; i < 9; i++) {
if (i == 1) {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 10);
} else if (i < 7) {
testTable.doClean("0000000" + i, cleanStats);
} else {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
}
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
if (i < 6) {
assertEquals(originalCommits, commitsAfterArchival);
} else if (i == 6) {
// 1,2,3,4,5,6 : after archival -> 1,5,6 (bcoz, 2,3,4,5 and 6 are clean commits and are eligible for archival)
List<HoodieInstant> expectedActiveInstants = new ArrayList<>();
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001")));
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000005", "00000006"), HoodieTimeline.CLEAN_ACTION));
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000003", "00000004"), HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival);
} else {
assertEquals(originalCommits, commitsAfterArchival);
}
}
}
@Test
public void testArchiveCompletedRollback() throws IOException {
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();
metaClient = HoodieTableMetaClient.reload(metaClient);
public void testArchiveRollbacksAndCleanTestTable() throws Exception {
boolean enableMetadata = false;
int minArchiveCommits = 2;
int maxArchiveCommits = 9;
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, minArchiveCommits, maxArchiveCommits, 2);
createCommitAndRollbackFile("6", "10", false);
createCommitAndRollbackFile("8", "11", false);
createCommitAndRollbackFile("7", "12", false);
HoodieInstant notArchivedInstant1 = new HoodieInstant(State.COMPLETED, "rollback", "12");
// trigger 1 commit to add lot of files so that future cleans can clean them up
testTable.doWriteOperation("00000001", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), Arrays.asList("p1", "p2"), 20);
createCommitAndRollbackFile("5", "13", false);
HoodieInstant notArchivedInstant2 = new HoodieInstant(State.COMPLETED, "rollback", "13");
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
List<HoodieInstant> notArchivedInstants = metaClient.getActiveTimeline().reload().getRollbackTimeline().getInstants().collect(Collectors.toList());
//There will be 2 * 2 files but due to TimelineLayoutV1 this will show as 2.
assertEquals(2, notArchivedInstants.size(), "Not archived instants should be 2");
assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2), "");
}
@Test
public void testArchiveCompletedShouldRetainMinInstantsIfInstantsGreaterThanMaxtoKeep() throws IOException {
int minInstants = 2;
int maxInstants = 10;
HoodieWriteConfig cfg =
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstants, maxInstants).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
for (int i = 0; i < maxInstants + 2; i++) {
createCleanMetadata(i + "", false);
Map<String, Integer> partitionToFileDeleteCount = new HashMap<>();
partitionToFileDeleteCount.put("p1", 1);
partitionToFileDeleteCount.put("p2", 1);
// we are triggering 10 clean commits. (1 is commit, 2 -> 11 is clean)
for (int i = 2; i <= (maxArchiveCommits + 2); i++) {
testTable.doClean((i > 9 ? ("000000") : ("0000000")) + i, partitionToFileDeleteCount);
}
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
archiveLog.archiveIfRequired(context);
assertEquals(minInstants, metaClient.getActiveTimeline().reload().getInstants().count());
}
@Test
public void testArchiveCompletedShouldNotArchiveIfInstantsLessThanMaxtoKeep() throws IOException {
int minInstants = 2;
int maxInstants = 10;
HoodieWriteConfig cfg =
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstants, maxInstants).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
for (int i = 0; i < maxInstants; i++) {
createCleanMetadata(i + "", false);
// we are triggering 7 commits and 7 rollbacks for the same
for (int i = 12; i <= (2 * maxArchiveCommits); i += 2) {
testTable.doWriteOperation("000000" + i, WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
testTable.doRollback("000000" + i, "000000" + (i + 1));
}
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(cfg, table);
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
archiveLog.archiveIfRequired(context);
assertEquals(maxInstants, metaClient.getActiveTimeline().reload().getInstants().count());
// out of 10 clean commits, 8 will be archived. 2 to 9. 10 and 11 will be active.
// wrt regular commits, there aren't 9 commits yet and so all of them will be active.
List<HoodieInstant> expectedActiveInstants = new ArrayList<>();
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000010", "00000011"), HoodieTimeline.CLEAN_ACTION));
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001", "00000012", "00000014", "00000016", "00000018")));
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000013", "00000015", "00000017", "00000019"), HoodieTimeline.ROLLBACK_ACTION));
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000002", "00000003", "00000004", "00000005", "00000006", "00000007", "00000008", "00000009"),
HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival);
}
@Test
public void testArchiveCompletedRollbackAndClean() throws IOException {
public void testArchiveCompletedRollbackAndClean() throws Exception {
init();
int minInstantsToKeep = 2;
int maxInstantsToKeep = 10;
HoodieWriteConfig cfg =
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build())
.build();
HoodieWriteConfig.newBuilder().withPath(basePath).withSchema(HoodieTestDataGenerator.TRIP_EXAMPLE_SCHEMA)
.withParallelism(2, 2).forTable("test-trip-table")
.withCompactionConfig(HoodieCompactionConfig.newBuilder().retainCommits(1).archiveCommitsWith(minInstantsToKeep, maxInstantsToKeep).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
int startInstant = 1;
@@ -612,12 +488,14 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
}
@Test
public void testArchiveInflightClean() throws IOException {
public void testArchiveInflightClean() throws Exception {
init();
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();
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())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
metaClient = HoodieTableMetaClient.reload(metaClient);
createCleanMetadata("10", false);
@@ -636,6 +514,137 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
assertEquals(notArchivedInstants, Arrays.asList(notArchivedInstant1, notArchivedInstant2, notArchivedInstant3), "");
}
@Test
public void testArchiveTableWithMetadataTableCompaction() throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(true, 2, 4, 7);
// min archival commits is 2 and max archival commits is 4. and so, after 5th commit, ideally archival should kick in. but max delta commits in metadata table is set to 6. and so
// archival will kick in only by 7th commit in datatable(1 commit for bootstrap + 6 commits from data table).
// and then 2nd compaction will take place
for (int i = 1; i < 6; i++) {
testTable.doWriteOperation("0000000" + i, WriteOperationType.UPSERT, i == 1 ? Arrays.asList("p1", "p2") : Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits, commitsAfterArchival);
}
// one more commit will trigger compaction in metadata table and will let archival move forward.
testTable.doWriteOperation("00000006", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
Pair<List<HoodieInstant>, List<HoodieInstant>> commitsList = archiveAndGetCommitsList(writeConfig);
List<HoodieInstant> originalCommits = commitsList.getKey();
List<HoodieInstant> commitsAfterArchival = commitsList.getValue();
// before archival 1,2,3,4,5,6
// after archival 5,6
assertEquals(originalCommits.size() - commitsAfterArchival.size(), 4);
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004")), getActiveCommitInstants(Arrays.asList("00000005", "00000006")), commitsAfterArchival);
// 3 more commits, 5 and 6 will be archived. but will not move after 6 since compaction has to kick in in metadata table.
testTable.doWriteOperation("00000007", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
testTable.doWriteOperation("00000008", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
commitsList = archiveAndGetCommitsList(writeConfig);
originalCommits = commitsList.getKey();
commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits, commitsAfterArchival);
// ideally, this will archive commits 5, 6, 7, but since compaction in metadata is until 6, only 5 and 6 will get archived,
testTable.doWriteOperation("00000009", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
commitsList = archiveAndGetCommitsList(writeConfig);
originalCommits = commitsList.getKey();
commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits.size() - commitsAfterArchival.size(), 2);
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006")),
getActiveCommitInstants(Arrays.asList("00000007", "00000008", "00000009")), commitsAfterArchival);
// and then 2nd compaction will take place at 12th commit
for (int i = 10; i < 13; i++) {
testTable.doWriteOperation("000000" + i, WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
commitsList = archiveAndGetCommitsList(writeConfig);
originalCommits = commitsList.getKey();
commitsAfterArchival = commitsList.getValue();
assertEquals(originalCommits, commitsAfterArchival);
}
// one more commit will trigger compaction in metadata table and will let archival move forward.
testTable.doWriteOperation("00000013", WriteOperationType.UPSERT, Collections.emptyList(), Arrays.asList("p1", "p2"), 2);
// trigger archival
commitsList = archiveAndGetCommitsList(writeConfig);
originalCommits = commitsList.getKey();
commitsAfterArchival = commitsList.getValue();
// before archival 5,6,7,8,9,10,11,12,13
// after archival 12,13
assertEquals(originalCommits.size() - commitsAfterArchival.size(), 5);
verifyArchival(getAllArchivedCommitInstants(Arrays.asList("00000001", "00000002", "00000003", "00000004", "00000005", "00000006", "00000007", "00000008",
"00000009", "00000010", "00000011")), getActiveCommitInstants(Arrays.asList("00000012", "00000013")), commitsAfterArchival);
}
private Pair<List<HoodieInstant>, List<HoodieInstant>> archiveAndGetCommitsList(HoodieWriteConfig writeConfig) throws IOException {
metaClient.reloadActiveTimeline();
HoodieTimeline timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> originalCommits = timeline.getInstants().collect(Collectors.toList());
HoodieTable table = HoodieSparkTable.create(writeConfig, context, metaClient);
HoodieTimelineArchiveLog archiveLog = new HoodieTimelineArchiveLog(writeConfig, table);
archiveLog.archiveIfRequired(context);
timeline = metaClient.getActiveTimeline().reload().getAllCommitsTimeline().filterCompletedInstants();
List<HoodieInstant> commitsAfterArchival = timeline.getInstants().collect(Collectors.toList());
return Pair.of(originalCommits, commitsAfterArchival);
}
private void verifyArchival(List<HoodieInstant> expectedArchivedInstants, List<HoodieInstant> expectedActiveInstants, List<HoodieInstant> commitsAfterArchival) {
Collections.sort(expectedActiveInstants, Comparator.comparing(HoodieInstant::getTimestamp));
Collections.sort(commitsAfterArchival, Comparator.comparing(HoodieInstant::getTimestamp));
assertEquals(expectedActiveInstants, commitsAfterArchival);
expectedArchivedInstants.forEach(entry -> assertFalse(commitsAfterArchival.contains(entry)));
HoodieArchivedTimeline archivedTimeline = new HoodieArchivedTimeline(metaClient);
List<HoodieInstant> actualArchivedInstants = archivedTimeline.getInstants().collect(Collectors.toList());
Collections.sort(actualArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp));
Collections.sort(expectedArchivedInstants, Comparator.comparing(HoodieInstant::getTimestamp));
assertEquals(actualArchivedInstants, expectedArchivedInstants);
HoodieTimeline timeline = metaClient.getActiveTimeline();
expectedArchivedInstants.forEach(entry -> {
// check safety
if (entry.getAction() != HoodieTimeline.ROLLBACK_ACTION) {
assertTrue(timeline.containsOrBeforeTimelineStarts(entry.getTimestamp()), "Archived commits should always be safe");
}
}
);
}
private List<HoodieInstant> getArchivedInstants(HoodieInstant instant) {
List<HoodieInstant> instants = new ArrayList<>();
if (instant.getAction() == HoodieTimeline.COMMIT_ACTION || instant.getAction() == HoodieTimeline.DELTA_COMMIT_ACTION || instant.getAction() == HoodieTimeline.CLEAN_ACTION) {
instants.add(new HoodieInstant(State.REQUESTED, instant.getAction(), instant.getTimestamp()));
}
instants.add(new HoodieInstant(State.INFLIGHT, instant.getAction(), instant.getTimestamp()));
instants.add(new HoodieInstant(State.COMPLETED, instant.getAction(), instant.getTimestamp()));
return instants;
}
private List<HoodieInstant> getAllArchivedCommitInstants(List<String> commitTimes) {
return getAllArchivedCommitInstants(commitTimes, HoodieTimeline.COMMIT_ACTION);
}
private List<HoodieInstant> getAllArchivedCommitInstants(List<String> commitTimes, String action) {
List<HoodieInstant> allInstants = new ArrayList<>();
commitTimes.forEach(entry -> allInstants.addAll(getArchivedInstants(new HoodieInstant(State.COMPLETED, action, entry))));
return allInstants;
}
private List<HoodieInstant> getActiveCommitInstants(List<String> commitTimes) {
return getActiveCommitInstants(commitTimes, HoodieTimeline.COMMIT_ACTION);
}
private List<HoodieInstant> getActiveCommitInstants(List<String> commitTimes, String action) {
List<HoodieInstant> allInstants = new ArrayList<>();
commitTimes.forEach(entry -> allInstants.add(new HoodieInstant(State.COMPLETED, action, entry)));
return allInstants;
}
private HoodieInstant createCleanMetadata(String instantTime, boolean inflightOnly) throws IOException {
HoodieCleanerPlan cleanerPlan = new HoodieCleanerPlan(new HoodieActionInstant("", "", ""), "", new HashMap<>(),
CleanPlanV2MigrationHandler.VERSION, new HashMap<>());
@@ -665,14 +674,14 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
HoodieTestTable.of(metaClient).addInflightRollback(rollbackTime);
} else {
HoodieRollbackMetadata hoodieRollbackMetadata = HoodieRollbackMetadata.newBuilder()
.setVersion(1)
.setStartRollbackTime(rollbackTime)
.setTotalFilesDeleted(1)
.setTimeTakenInMillis(1000)
.setCommitsRollback(Collections.singletonList(commitToRollback))
.setPartitionMetadata(Collections.emptyMap())
.setInstantsRollback(Collections.emptyList())
.build();
.setVersion(1)
.setStartRollbackTime(rollbackTime)
.setTotalFilesDeleted(1)
.setTimeTakenInMillis(1000)
.setCommitsRollback(Collections.singletonList(commitToRollback))
.setPartitionMetadata(Collections.emptyMap())
.setInstantsRollback(Collections.emptyList())
.build();
HoodieTestTable.of(metaClient).addRollback(rollbackTime, hoodieRollbackMetadata);
}
return new HoodieInstant(inflight, "rollback", rollbackTime);

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.io.storage.row;
import org.apache.hudi.client.HoodieInternalWriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.testutils.HoodieTestDataGenerator;
@@ -168,7 +169,8 @@ public class TestHoodieRowCreateHandle extends HoodieClientTestHarness {
@Test
public void testInstantiationFailure() throws IOException {
// init config and table
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withPath("/dummypath/abc/").build();
HoodieWriteConfig cfg = SparkDatasetTestUtils.getConfigBuilder(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.withPath("/dummypath/abc/").build();
HoodieTable table = HoodieSparkTable.create(cfg, context, metaClient);
try {

View File

@@ -266,6 +266,7 @@ public class TestCleaner extends HoodieClientTestBase {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(maxVersions).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg);) {
@@ -298,7 +299,7 @@ public class TestCleaner extends HoodieClientTestBase {
.map(e -> Pair.of(e.getKey().getPartitionPath(), e.getValue())).collect(Collectors.toList());
HoodieCompactionPlan compactionPlan =
CompactionUtils.buildFromFileSlices(partitionFileSlicePairs, Option.empty(), Option.empty());
List<String> instantTimes = makeIncrementalCommitTimes(9);
List<String> instantTimes = makeIncrementalCommitTimes(9, 1, 10);
String compactionTime = instantTimes.get(0);
table.getActiveTimeline().saveToCompactionRequested(
new HoodieInstant(State.REQUESTED, HoodieTimeline.COMPACTION_ACTION, compactionTime),
@@ -435,6 +436,7 @@ public class TestCleaner extends HoodieClientTestBase {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
@@ -511,6 +513,7 @@ public class TestCleaner extends HoodieClientTestBase {
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(maxCommits).build())
.withParallelism(1, 1).withBulkInsertParallelism(1).withFinalizeWriteParallelism(1).withDeleteParallelism(1)
.withConsistencyGuardConfig(ConsistencyGuardConfig.newBuilder().withConsistencyCheckEnabled(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
SparkRDDWriteClient client = getHoodieWriteClient(cfg);
@@ -639,7 +642,7 @@ public class TestCleaner extends HoodieClientTestBase {
public void testKeepLatestFileVersions(Boolean enableBootstrapSourceClean) throws Exception {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanBootstrapBaseFileEnabled(enableBootstrapSourceClean)
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
@@ -731,7 +734,7 @@ public class TestCleaner extends HoodieClientTestBase {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_FILE_VERSIONS).retainFileVersions(1).build())
.build();
@@ -769,7 +772,7 @@ public class TestCleaner extends HoodieClientTestBase {
HoodieWriteConfig config =
HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(1).build())
.build();
@@ -809,7 +812,7 @@ public class TestCleaner extends HoodieClientTestBase {
@Test
public void testCleanWithReplaceCommits() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();
@@ -1116,7 +1119,7 @@ public class TestCleaner extends HoodieClientTestBase {
@MethodSource("argumentsForTestKeepLatestCommits")
public void testKeepLatestCommits(boolean simulateFailureRetry, boolean enableIncrementalClean, boolean enableBootstrapSourceClean) throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withIncrementalCleaningMode(enableIncrementalClean)
.withFailedWritesCleaningPolicy(HoodieFailedWritesCleaningPolicy.EAGER)
@@ -1295,7 +1298,7 @@ public class TestCleaner extends HoodieClientTestBase {
@Test
public void testCleaningWithZeroPartitionPaths() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();
@@ -1317,7 +1320,7 @@ public class TestCleaner extends HoodieClientTestBase {
@Test
public void testKeepLatestCommitsWithPendingCompactions() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().withAssumeDatePartitioning(true).enable(false).build())
.withCompactionConfig(HoodieCompactionConfig.newBuilder()
.withCleanerPolicy(HoodieCleaningPolicy.KEEP_LATEST_COMMITS).retainCommits(2).build())
.build();

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.table;
import org.apache.hudi.client.HoodieReadClient;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieCommitMetadata;
@@ -193,7 +194,7 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
@ValueSource(booleans = {true, false})
public void testLogFileCountsAfterCompaction(boolean populateMetaFields) throws Exception {
// insert 100 records
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true);
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(true).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build());
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig config = cfgBuilder.build();
@@ -524,6 +525,7 @@ public class TestHoodieMergeOnReadTable extends SparkClientFunctionalTestHarness
JavaRDD<HoodieRecord> deleteRDD = jsc().parallelize(fewRecordsForDelete, 1);
// initialize partitioner
hoodieTable.getHoodieView().sync();
AbstractSparkDeltaCommitActionExecutor actionExecutor = new SparkDeleteDeltaCommitActionExecutor(context(), cfg, hoodieTable,
newDeleteTime, deleteRDD);
actionExecutor.getUpsertPartitioner(new WorkloadProfile(buildProfile(deleteRDD)));

View File

@@ -52,7 +52,7 @@ public class TestAsyncCompaction extends CompactionTestBase {
private HoodieWriteConfig getConfig(Boolean autoCommit) {
return getConfigBuilder(autoCommit)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).validate(true).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build())
.build();
}

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.table.action.compact;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.fs.FSUtils;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieRecord;
@@ -159,7 +160,10 @@ public class TestHoodieCompactor extends HoodieClientTestHarness {
@Test
public void testWriteStatusContentsAfterCompaction() throws Exception {
// insert 100 records
HoodieWriteConfig config = getConfig();
HoodieWriteConfig config = getConfigBuilder()
.withCompactionConfig(HoodieCompactionConfig.newBuilder().withMaxNumDeltaCommitsBeforeCompaction(1).build())
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build())
.build();
try (SparkRDDWriteClient writeClient = getHoodieWriteClient(config)) {
String newCommitTime = "100";
writeClient.startCommitWithTime(newCommitTime);

View File

@@ -20,6 +20,7 @@ package org.apache.hudi.table.action.rollback;
import org.apache.hudi.avro.model.HoodieRollbackPartitionMetadata;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieFileGroup;
import org.apache.hudi.common.model.HoodieLogFile;
@@ -157,7 +158,7 @@ public class TestMergeOnReadRollbackActionExecutor extends HoodieClientRollbackT
@Test
public void testRollbackWhenFirstCommitFail() throws Exception {
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).build();
HoodieWriteConfig config = HoodieWriteConfig.newBuilder().withPath(basePath).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(config)) {
client.startCommitWithTime("001");

View File

@@ -131,6 +131,7 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
}
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient);
hoodieTable.getHoodieView().sync();
FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable);
// expect 2 base files for each partition
assertEquals(dataGen.getPartitionPaths().length * 2, allFiles.length);
@@ -146,6 +147,7 @@ class TestHoodieSparkMergeOnReadTableClustering extends SparkClientFunctionalTes
metaClient = HoodieTableMetaClient.reload(metaClient);
final HoodieTable clusteredTable = HoodieSparkTable.create(cfg, context(), metaClient);
clusteredTable.getHoodieView().sync();
Stream<HoodieBaseFile> dataFilesToRead = Arrays.stream(dataGen.getPartitionPaths())
.flatMap(p -> clusteredTable.getBaseFileOnlyView().getLatestBaseFiles(p));
// verify there should be only one base file per partition after clustering.

View File

@@ -20,6 +20,7 @@
package org.apache.hudi.table.functional;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.HoodieFileFormat;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieTableType;
@@ -83,7 +84,7 @@ public class TestHoodieSparkMergeOnReadTableIncrementalRead extends SparkClientF
Properties props = new Properties();
props.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieFileFormat.PARQUET.toString());
HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, props);
HoodieWriteConfig cfg = getConfig(true);
HoodieWriteConfig cfg = getConfigBuilder(true).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
try (SparkRDDWriteClient client = getHoodieWriteClient(cfg)) {
/*

View File

@@ -109,6 +109,7 @@ public class TestHoodieSparkMergeOnReadTableInsertUpdateDelete extends SparkClie
client.compact(compactionCommitTime);
HoodieTable hoodieTable = HoodieSparkTable.create(cfg, context(), metaClient);
hoodieTable.getHoodieView().sync();
FileStatus[] allFiles = listAllBaseFilesInPath(hoodieTable);
HoodieTableFileSystemView tableView = getHoodieTableFileSystemView(metaClient, hoodieTable.getCompletedCommitsTimeline(), allFiles);
Stream<HoodieBaseFile> dataFilesToRead = tableView.getLatestBaseFiles();
@@ -238,6 +239,7 @@ public class TestHoodieSparkMergeOnReadTableInsertUpdateDelete extends SparkClie
writeClient.commit(newCommitTime, statuses);
HoodieTable table = HoodieSparkTable.create(config, context(), metaClient);
table.getHoodieView().sync();
TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView();
long numLogFiles = 0;

View File

@@ -21,6 +21,7 @@ package org.apache.hudi.table.functional;
import org.apache.hudi.client.SparkRDDWriteClient;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.config.HoodieMetadataConfig;
import org.apache.hudi.common.model.FileSlice;
import org.apache.hudi.common.model.HoodieBaseFile;
import org.apache.hudi.common.model.HoodieFileGroup;
@@ -51,9 +52,8 @@ import org.apache.hadoop.fs.FileStatus;
import org.apache.hadoop.fs.Path;
import org.apache.spark.api.java.JavaRDD;
import org.junit.jupiter.api.Tag;
import org.junit.jupiter.api.Test;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.Arguments;
import org.junit.jupiter.params.provider.MethodSource;
import org.junit.jupiter.params.provider.ValueSource;
import java.io.File;
@@ -136,23 +136,14 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
}
}
private static Stream<Arguments> testRollbackWithDeltaAndCompactionCommit() {
return Stream.of(
Arguments.of(true, true),
Arguments.of(true, false),
Arguments.of(false, true),
Arguments.of(false, false)
);
}
@ParameterizedTest
@MethodSource
void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers, boolean populateMetaFields) throws Exception {
@ValueSource(booleans = {true, false})
void testRollbackWithDeltaAndCompactionCommit(boolean rollbackUsingMarkers) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false, rollbackUsingMarkers, HoodieIndex.IndexType.SIMPLE);
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
addConfigsForPopulateMetaFields(cfgBuilder, true);
HoodieWriteConfig cfg = cfgBuilder.build();
Properties properties = populateMetaFields ? new Properties() : getPropertiesForKeyGen();
Properties properties = new Properties();
properties.setProperty(HoodieTableConfig.BASE_FILE_FORMAT.key(), HoodieTableConfig.BASE_FILE_FORMAT.defaultValue().toString());
HoodieTableMetaClient metaClient = getHoodieMetaClient(HoodieTableType.MERGE_ON_READ, properties);
@@ -300,10 +291,10 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
}
}
@ParameterizedTest
@ValueSource(booleans = {true, false})
void testMultiRollbackWithDeltaAndCompactionCommit(boolean populateMetaFields) throws Exception {
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false);
@Test
void testMultiRollbackWithDeltaAndCompactionCommit() throws Exception {
boolean populateMetaFields = true;
HoodieWriteConfig.Builder cfgBuilder = getConfigBuilder(false).withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build());
addConfigsForPopulateMetaFields(cfgBuilder, populateMetaFields);
HoodieWriteConfig cfg = cfgBuilder.build();
@@ -352,7 +343,9 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
*/
newCommitTime = "002";
// WriteClient with custom config (disable small file handling)
try (SparkRDDWriteClient nClient = getHoodieWriteClient(getHoodieWriteConfigWithSmallFileHandlingOff(populateMetaFields))) {
HoodieWriteConfig smallFileWriteConfig = getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields)
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
try (SparkRDDWriteClient nClient = getHoodieWriteClient(smallFileWriteConfig)) {
nClient.startCommitWithTime(newCommitTime);
List<HoodieRecord> copyOfRecords = new ArrayList<>(records);
@@ -463,6 +456,10 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
}
private HoodieWriteConfig getHoodieWriteConfigWithSmallFileHandlingOff(boolean populateMetaFields) {
return getHoodieWriteConfigWithSmallFileHandlingOffBuilder(populateMetaFields).build();
}
private HoodieWriteConfig.Builder getHoodieWriteConfigWithSmallFileHandlingOffBuilder(boolean populateMetaFields) {
HoodieWriteConfig.Builder cfgBuilder = HoodieWriteConfig.newBuilder().withPath(basePath()).withSchema(TRIP_EXAMPLE_SCHEMA).withParallelism(2, 2)
.withDeleteParallelism(2)
.withAutoCommit(false)
@@ -474,7 +471,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
if (!populateMetaFields) {
addConfigsForPopulateMetaFields(cfgBuilder, false);
}
return cfgBuilder.build();
return cfgBuilder;
}
@ParameterizedTest
@@ -592,6 +589,7 @@ public class TestHoodieSparkMergeOnReadTableRollback extends SparkClientFunction
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(config, context(), metaClient);
table.getHoodieView().sync();
TableFileSystemView.SliceView tableRTFileSystemView = table.getSliceView();
long numLogFiles = 0;

View File

@@ -46,6 +46,7 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieMetadataException;
import org.apache.hudi.index.HoodieIndex;
import org.apache.hudi.keygen.SimpleKeyGenerator;
import org.apache.hudi.metadata.FileSystemBackedTableMetadata;
import org.apache.hudi.metadata.HoodieBackedTableMetadataWriter;
import org.apache.hudi.metadata.HoodieTableMetadata;
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
@@ -453,10 +454,14 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
String metadataTableBasePath, boolean doFullValidation) throws IOException {
HoodieTableMetadata tableMetadata = metadata(writeConfig, context);
assertNotNull(tableMetadata, "MetadataReader should have been initialized");
if (!writeConfig.isMetadataTableEnabled() || !writeConfig.getMetadataConfig().validateFileListingMetadata()) {
if (!writeConfig.isMetadataTableEnabled()) {
return;
}
if (!tableMetadata.getSyncedInstantTime().isPresent() || tableMetadata instanceof FileSystemBackedTableMetadata) {
throw new IllegalStateException("Metadata should have synced some commits or tableMetadata should not be an instance "
+ "of FileSystemBackedTableMetadata");
}
assertEquals(inflightCommits, testTable.inflightCommits());
HoodieTimer timer = new HoodieTimer().startTimer();
@@ -476,7 +481,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
// Files within each partition should match
metaClient = HoodieTableMetaClient.reload(metaClient);
HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext);
HoodieTable table = HoodieSparkTable.create(writeConfig, engineContext, true);
TableFileSystemView tableView = table.getHoodieView();
List<String> fullPartitionPaths = fsPartitions.stream().map(partition -> basePath + "/" + partition).collect(Collectors.toList());
Map<String, FileStatus[]> partitionToFilesMap = tableMetadata.getAllFilesInPartitions(fullPartitionPaths);
@@ -497,7 +502,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
}
public void syncTableMetadata(HoodieWriteConfig writeConfig) {
if (!writeConfig.getMetadataConfig().enableSync()) {
if (!writeConfig.getMetadataConfig().enabled()) {
return;
}
// Open up the metadata table again, for syncing
@@ -518,8 +523,8 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
clientConfig.getSpillableMapBasePath());
}
private void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView,
Map<String, FileStatus[]> partitionToFilesMap, String partition) throws IOException {
protected void validateFilesPerPartition(HoodieTestTable testTable, HoodieTableMetadata tableMetadata, TableFileSystemView tableView,
Map<String, FileStatus[]> partitionToFilesMap, String partition) throws IOException {
Path partitionPath;
if (partition.equals("")) {
// Should be the non-partitioned case
@@ -537,8 +542,6 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
Collections.sort(fsFileNames);
Collections.sort(metadataFilenames);
assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length);
if ((fsFileNames.size() != metadataFilenames.size()) || (!fsFileNames.equals(metadataFilenames))) {
LOG.info("*** File system listing = " + Arrays.toString(fsFileNames.toArray()));
LOG.info("*** Metadata listing = " + Arrays.toString(metadataFilenames.toArray()));
@@ -554,6 +557,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
}
}
}
assertEquals(fsStatuses.length, partitionToFilesMap.get(basePath + "/" + partition).length);
// Block sizes should be valid
Arrays.stream(metaStatuses).forEach(s -> assertTrue(s.getBlockSize() > 0));
@@ -586,8 +590,6 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
HoodieWriteConfig metadataWriteConfig = metadataWriter.getWriteConfig();
assertFalse(metadataWriteConfig.isMetadataTableEnabled(), "No metadata table for metadata table");
// Metadata table should be in sync with the dataset
assertTrue(metadata(writeConfig, engineContext).isInSync());
HoodieTableMetaClient metadataMetaClient = HoodieTableMetaClient.builder().setConf(hadoopConf).setBasePath(metadataTableBasePath).build();
// Metadata table is MOR
@@ -601,7 +603,7 @@ public abstract class HoodieClientTestHarness extends HoodieCommonTestHarness im
// Cannot use FSUtils.getAllFoldersWithPartitionMetaFile for this as that function filters all directory
// in the .hoodie folder.
List<String> metadataTablePartitions = FSUtils.getAllPartitionPaths(engineContext, HoodieTableMetadata.getMetadataTableBasePath(basePath),
false, false, false);
false, false);
Assertions.assertEquals(MetadataPartitionType.values().length, metadataTablePartitions.size());
// Metadata table should automatically compact and clean