1
0

[HUDI-2595] Fixing metadata table updates such that only regular writes from data table can trigger table services in metadata table (#3900)

This commit is contained in:
Sivabalan Narayanan
2021-11-08 22:12:32 -05:00
committed by GitHub
parent 7aaf47e716
commit 6d109c6de5
22 changed files with 170 additions and 59 deletions

View File

@@ -184,11 +184,12 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
HoodieTable table = createTable(config, hadoopConf);
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(stats, partitionToReplaceFileIds,
extraMetadata, operationType, config.getWriteSchema(), commitActionType);
HoodieInstant inflightInstant = new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime);
HeartbeatUtils.abortIfHeartbeatExpired(instantTime, table, heartbeatClient, config);
this.txnManager.beginTransaction(Option.of(new HoodieInstant(State.INFLIGHT, table.getMetaClient().getCommitActionType(), instantTime)),
this.txnManager.beginTransaction(Option.of(inflightInstant),
lastCompletedTxnAndMetadata.isPresent() ? Option.of(lastCompletedTxnAndMetadata.get().getLeft()) : Option.empty());
try {
preCommit(instantTime, metadata);
preCommit(inflightInstant, metadata);
commit(table, commitActionType, instantTime, metadata, stats);
postCommit(table, metadata, instantTime, extraMetadata);
LOG.info("Committed " + instantTime);
@@ -244,14 +245,15 @@ public abstract class AbstractHoodieWriteClient<T extends HoodieRecordPayload, I
/**
* Any pre-commit actions like conflict resolution or updating metadata table goes here.
* @param instantTime commit instant time.
* @param inflightInstant instant of inflight operation.
* @param metadata commit metadata for which pre commit is being invoked.
*/
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
// Create a Hoodie table after starting the transaction which encapsulated the commits and files visible.
// Important to create this after the lock to ensure latest commits show up in the timeline without need for reload
HoodieTable table = createTable(config, hadoopConf);
table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, instantTime));
table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, inflightInstant.getTimestamp(),
table.isTableServiceAction(inflightInstant.getAction())));
}
/**

View File

@@ -409,7 +409,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
});
LOG.info("Committing " + partitionToFileStatus.size() + " partitions and " + stats[0] + " files to metadata");
update(commitMetadata, createInstantTime);
update(commitMetadata, createInstantTime, false);
return true;
}
@@ -523,23 +523,24 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
* @param instantTime instant time of interest.
* @param convertMetadataFunction converter function to convert the respective metadata to List of HoodieRecords to be written to metadata table.
* @param <T> type of commit metadata.
* @param canTriggerTableService true if table services can be triggered. false otherwise.
*/
private <T> void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction) {
private <T> void processAndCommit(String instantTime, ConvertMetadataFunction convertMetadataFunction, boolean canTriggerTableService) {
if (enabled && metadata != null) {
List<HoodieRecord> records = convertMetadataFunction.convertMetadata();
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime, canTriggerTableService);
}
}
/**
* Update from {@code HoodieCommitMetadata}.
*
* @param commitMetadata {@code HoodieCommitMetadata}
* @param instantTime Timestamp at which the commit was performed
* @param isTableServiceAction {@code true} if commit metadata is pertaining to a table service. {@code false} otherwise.
*/
@Override
public void update(HoodieCommitMetadata commitMetadata, String instantTime) {
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime));
public void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction) {
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(commitMetadata, instantTime), !isTableServiceAction);
}
/**
@@ -550,7 +551,8 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
*/
@Override
public void update(HoodieCleanMetadata cleanMetadata, String instantTime) {
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime));
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(cleanMetadata, instantTime),
false);
}
/**
@@ -562,7 +564,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
@Override
public void update(HoodieRestoreMetadata restoreMetadata, String instantTime) {
processAndCommit(instantTime, () -> HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(),
restoreMetadata, instantTime, metadata.getSyncedInstantTime()));
restoreMetadata, instantTime, metadata.getSyncedInstantTime()), false);
}
/**
@@ -588,7 +590,7 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
List<HoodieRecord> records = HoodieTableMetadataUtil.convertMetadataToRecords(metadataMetaClient.getActiveTimeline(), rollbackMetadata, instantTime,
metadata.getSyncedInstantTime(), wasSynced);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime);
commit(records, MetadataPartitionType.FILES.partitionPath(), instantTime, false);
}
}
@@ -601,12 +603,12 @@ public abstract class HoodieBackedTableMetadataWriter implements HoodieTableMeta
/**
* Commit the {@code HoodieRecord}s to Metadata Table as a new delta-commit.
*
* @param records The list of records to be written.
* @param records The list of records to be written.
* @param partitionName The partition to which the records are to be written.
* @param instantTime The timestamp to use for the deltacommit.
* @param canTriggerTableService true if table services can be scheduled and executed. false otherwise.
*/
protected abstract void commit(List<HoodieRecord> records, String partitionName, String instantTime);
protected abstract void commit(List<HoodieRecord> records, String partitionName, String instantTime, boolean canTriggerTableService);
/**
* Perform a compaction on the Metadata Table.

View File

@@ -34,8 +34,10 @@ public interface HoodieTableMetadataWriter extends Serializable, AutoCloseable {
* Update the metadata table due to a COMMIT operation.
* @param commitMetadata commit metadata of the operation of interest.
* @param instantTime instant time of the commit.
* @param isTableServiceAction true if caller is a table service. false otherwise. Only regular write operations can trigger metadata table services and this argument
* will assist in this.
*/
void update(HoodieCommitMetadata commitMetadata, String instantTime);
void update(HoodieCommitMetadata commitMetadata, String instantTime, boolean isTableServiceAction);
/**
* Update the metadata table due to a CLEAN operation.

View File

@@ -738,6 +738,13 @@ public abstract class HoodieTable<T extends HoodieRecordPayload, I, K, O> implem
return getMetadataWriter(Option.empty());
}
/**
* Check if action type is a table service.
* @param actionType action type of interest.
* @return true if action represents a table service. false otherwise.
*/
public abstract boolean isTableServiceAction(String actionType);
/**
* Get Table metadata writer.
*

View File

@@ -56,8 +56,8 @@ public abstract class BaseActionExecutor<T extends HoodieRecordPayload, I, K, O,
* Writes commits metadata to table metadata.
* @param metadata commit metadata of interest.
*/
protected final void writeTableMetadata(HoodieCommitMetadata metadata) {
table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime));
protected final void writeTableMetadata(HoodieCommitMetadata metadata, String actionType) {
table.getMetadataWriter().ifPresent(w -> w.update(metadata, instantTime, table.isTableServiceAction(actionType)));
}
/**

View File

@@ -77,7 +77,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
HoodieCommitMetadata commitMetadata = super.doWriteOperation(commitTime, operationType, newPartitionsToAdd,
partitionToFilesNameLengthMap, bootstrap, createInflightCommit);
if (writer != null && !createInflightCommit) {
writer.update(commitMetadata, commitTime);
writer.update(commitMetadata, commitTime, false);
}
return commitMetadata;
}
@@ -86,7 +86,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException {
super.moveInflightCommitToComplete(instantTime, metadata);
if (writer != null) {
writer.update(metadata, instantTime);
writer.update(metadata, instantTime, false);
}
return this;
}
@@ -94,7 +94,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
public HoodieTestTable moveInflightCommitToComplete(String instantTime, HoodieCommitMetadata metadata, boolean ignoreWriter) throws IOException {
super.moveInflightCommitToComplete(instantTime, metadata);
if (!ignoreWriter && writer != null) {
writer.update(metadata, instantTime);
writer.update(metadata, instantTime, false);
}
return this;
}
@@ -103,7 +103,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
public HoodieTestTable moveInflightCompactionToComplete(String instantTime, HoodieCommitMetadata metadata) throws IOException {
super.moveInflightCompactionToComplete(instantTime, metadata);
if (writer != null) {
writer.update(metadata, instantTime);
writer.update(metadata, instantTime, true);
}
return this;
}
@@ -120,7 +120,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
public HoodieTestTable addCompaction(String instantTime, HoodieCommitMetadata commitMetadata) throws Exception {
super.addCompaction(instantTime, commitMetadata);
if (writer != null) {
writer.update(commitMetadata, instantTime);
writer.update(commitMetadata, instantTime, true);
}
return this;
}
@@ -151,7 +151,7 @@ public class HoodieMetadataTestTable extends HoodieTestTable {
HoodieReplaceCommitMetadata completeReplaceMetadata) throws Exception {
super.addReplaceCommit(instantTime, requestedReplaceMetadata, inflightReplaceMetadata, completeReplaceMetadata);
if (writer != null) {
writer.update(completeReplaceMetadata, instantTime);
writer.update(completeReplaceMetadata, instantTime, true);
}
return this;
}

View File

@@ -258,10 +258,10 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
}
@Override
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
this.metadataWriterOption.ifPresent(w -> {
w.initTableMetadata(); // refresh the timeline
w.update(metadata, instantTime);
w.update(metadata, inflightInstant.getTimestamp(), getHoodieTable().isTableServiceAction(inflightInstant.getAction()));
});
}
@@ -406,7 +406,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
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()));
table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), table.isTableServiceAction(hoodieInstant.getAction())));
} finally {
this.txnManager.endTransaction();
}

View File

@@ -90,7 +90,7 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
}
@Override
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime, boolean canTriggerTableService) {
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
List<HoodieRecord> recordList = prepRecords(records, partitionName, 1);
@@ -125,8 +125,10 @@ public class FlinkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
// reload timeline
metadataMetaClient.reloadActiveTimeline();
compactIfNecessary(writeClient, instantTime);
doClean(writeClient, instantTime);
if (canTriggerTableService) {
compactIfNecessary(writeClient, instantTime);
doClean(writeClient, instantTime);
}
}
// Update total size of the metadata and count of base/log files

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieNotSupportedException;
@@ -84,6 +85,11 @@ public class HoodieFlinkCopyOnWriteTable<T extends HoodieRecordPayload>
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.COMMIT_ACTION);
}
/**
* Upsert a batch of new records into Hoodie table at the supplied instantTime.
*

View File

@@ -27,6 +27,7 @@ import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.common.util.ValidationUtils;
import org.apache.hudi.config.HoodieWriteConfig;
@@ -54,6 +55,11 @@ public class HoodieFlinkMergeOnReadTable<T extends HoodieRecordPayload>
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.DELTA_COMMIT_ACTION);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsert(
HoodieEngineContext context,

View File

@@ -147,7 +147,7 @@ public abstract class BaseFlinkCommitActionExecutor<T extends HoodieRecordPayloa
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
writeTableMetadata(metadata);
writeTableMetadata(metadata, actionType);
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));

View File

@@ -35,6 +35,7 @@ import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieInstant;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieNotSupportedException;
@@ -66,6 +67,11 @@ public class HoodieJavaCopyOnWriteTable<T extends HoodieRecordPayload> extends H
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.COMMIT_ACTION);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsert(HoodieEngineContext context,
String instantTime,

View File

@@ -24,6 +24,7 @@ import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.timeline.HoodieTimeline;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.action.HoodieWriteMetadata;
@@ -37,6 +38,11 @@ public class HoodieJavaMergeOnReadTable<T extends HoodieRecordPayload> extends H
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.DELTA_COMMIT_ACTION);
}
@Override
public HoodieWriteMetadata<List<WriteStatus>> upsertPrepped(HoodieEngineContext context,
String instantTime,

View File

@@ -208,7 +208,7 @@ public abstract class BaseJavaCommitActionExecutor<T extends HoodieRecordPayload
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
writeTableMetadata(metadata);
writeTableMetadata(metadata, actionType);
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));

View File

@@ -410,9 +410,10 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieInstant hoodieInstant) {
try {
this.txnManager.beginTransaction(Option.of(hoodieInstant), Option.empty());
boolean isTableServiceAction = table.isTableServiceAction(hoodieInstant.getAction());
// 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()));
table.getMetadataWriter().ifPresent(w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction));
} finally {
this.txnManager.endTransaction();
}
@@ -478,13 +479,14 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
}
@Override
protected void preCommit(String instantTime, HoodieCommitMetadata metadata) {
protected void preCommit(HoodieInstant inflightInstant, HoodieCommitMetadata metadata) {
// Create a Hoodie table after startTxn which encapsulated the commits and files visible.
// Important to create this after the lock to ensure latest commits show up in the timeline without need for reload
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));
table.getMetadataWriter().ifPresent(w -> ((HoodieTableMetadataWriter)w).update(metadata, inflightInstant.getTimestamp(),
table.isTableServiceAction(inflightInstant.getAction())));
}
@Override

View File

@@ -103,7 +103,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
}
@Override
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime) {
protected void commit(List<HoodieRecord> records, String partitionName, String instantTime, boolean canTriggerTableService) {
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
@@ -132,8 +132,10 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
// reload timeline
metadataMetaClient.reloadActiveTimeline();
compactIfNecessary(writeClient, instantTime);
doClean(writeClient, instantTime);
if (canTriggerTableService) {
compactIfNecessary(writeClient, instantTime);
doClean(writeClient, instantTime);
}
}
// Update total size of the metadata and count of base/log files

View File

@@ -101,6 +101,11 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload>
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.COMMIT_ACTION);
}
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
return new SparkUpsertCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();

View File

@@ -80,6 +80,11 @@ public class HoodieSparkMergeOnReadTable<T extends HoodieRecordPayload> extends
super(config, context, metaClient);
}
@Override
public boolean isTableServiceAction(String actionType) {
return !actionType.equals(HoodieTimeline.DELTA_COMMIT_ACTION);
}
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> upsert(HoodieEngineContext context, String instantTime, JavaRDD<HoodieRecord<T>> records) {
return new SparkUpsertDeltaCommitActionExecutor<>((HoodieSparkEngineContext) context, config, this, instantTime, records).execute();

View File

@@ -248,7 +248,7 @@ public class SparkBootstrapCommitActionExecutor<T extends HoodieRecordPayload<T>
metadata.addMetadata(HoodieCommitMetadata.SCHEMA_KEY, getSchemaToStoreInCommit());
metadata.setOperationType(operationType);
writeTableMetadata(metadata);
writeTableMetadata(metadata, actionType);
try {
activeTimeline.saveAsComplete(new HoodieInstant(true, actionType, instantTime),

View File

@@ -267,7 +267,7 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
HoodieCommitMetadata metadata = CommitUtils.buildMetadata(writeStats, result.getPartitionToReplaceFileIds(),
extraMetadata, operationType, getSchemaToStoreInCommit(), getCommitActionType());
writeTableMetadata(metadata);
writeTableMetadata(metadata, actionType);
activeTimeline.saveAsComplete(new HoodieInstant(true, getCommitActionType(), instantTime),
Option.of(metadata.toJsonString().getBytes(StandardCharsets.UTF_8)));
LOG.info("Committed " + instantTime);

View File

@@ -273,6 +273,37 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
validateMetadata(testTable, emptyList(), true);
}
/**
* Tests that table services in data table won't trigger table services in metadata table.
* @throws Exception
*/
@Test
public void testMetadataTableServices() throws Exception {
HoodieTableType tableType = COPY_ON_WRITE;
init(tableType, false);
writeConfig = getWriteConfigBuilder(true, true, false)
.withMetadataConfig(HoodieMetadataConfig.newBuilder()
.enable(true)
.enableFullScan(true)
.enableMetrics(false)
.withMaxNumDeltaCommitsBeforeCompaction(3) // after 3 delta commits for regular writer operations, compaction should kick in.
.build()).build();
initWriteConfigAndMetatableWriter(writeConfig, true);
doWriteOperation(testTable, "0000001", INSERT);
doCleanAndValidate(testTable, "0000003", Arrays.asList("0000001"));
HoodieTableMetadata tableMetadata = metadata(writeConfig, context);
// since clean was the last commit, table servives should not get triggered in metadata table.
assertFalse(tableMetadata.getLatestCompactionTime().isPresent());
doWriteOperation(testTable, "0000004", UPSERT);
// this should have triggered compaction in metadata table
tableMetadata = metadata(writeConfig, context);
assertTrue(tableMetadata.getLatestCompactionTime().isPresent());
assertEquals(tableMetadata.getLatestCompactionTime().get(), "0000004001");
}
/**
* Test rollback of various table operations sync to Metadata Table correctly.
*/
@@ -467,7 +498,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build())) {
// Write 1
String commitTime = "0000001";
@@ -501,7 +533,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
init(tableType);
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext,
getWriteConfigBuilder(true, true, false).withRollbackUsingMarkers(false).build())) {
// Write 1 (Bulk insert)
String newCommitTime = "0000001";
@@ -685,8 +718,8 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
/**
* Lets say clustering commit succeeded in metadata table, but failed before committing to datatable.
* Next time, when clustering kicks in, hudi will rollback pending clustering and re-attempt the clustering with same instant time.
* So, this test ensures the 2nd attempt succeeds with metadata enabled.
* Next time, when clustering kicks in, hudi will rollback pending clustering (in data table) and re-attempt the clustering with same
* instant time. So, this test ensures the 2nd attempt succeeds with metadata enabled.
* This is applicable to any table service where instant time is fixed. So, how many ever times the operation fails, re attempt will
* be made with same commit time.
* Tests uses clustering to test out the scenario.

View File

@@ -305,18 +305,20 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
}
}
// 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)
// move inflight compaction to complete and add one regular write commit. archival should archive more commits.
// an extra one commit is required, bcoz compaction in data table will not trigger table services in metadata table.
// before this move, timeline : 2_inflight_compaction, 3,4,5,6,7.
// after this move: 6,7,8 (2,3,4,5 will be archived)
testTable.moveInflightCompactionToComplete("00000002", inflightCompactionMetadata);
testTable.doWriteOperation("00000008", WriteOperationType.UPSERT, Arrays.asList("p1", "p2"), 2);
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);
List<HoodieInstant> archivedInstants = getAllArchivedCommitInstants(Arrays.asList("00000001", "00000003", "00000004", "00000005", "00000006"), 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);
verifyArchival(archivedInstants, getActiveCommitInstants(Arrays.asList("00000007", "00000008"), HoodieTimeline.DELTA_COMMIT_ACTION), commitsAfterArchival);
}
@Test
@@ -379,7 +381,8 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
public void testArchiveTableWithCleanCommits(boolean enableMetadata) throws Exception {
HoodieWriteConfig writeConfig = initTestTableAndGetWriteConfig(enableMetadata, 2, 4, 2);
// 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.
// 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<>();
@@ -400,13 +403,35 @@ public class TestHoodieTimelineArchiveLog extends HoodieClientTestHarness {
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);
if (!enableMetadata) {
// 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 {
// with metadata enabled, archival in data table is fenced based on compaction in metadata table. Clean commits in data table will not trigger compaction in
// metadata table.
List<HoodieInstant> expectedActiveInstants = new ArrayList<>();
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000001")));
expectedActiveInstants.addAll(getActiveCommitInstants(Arrays.asList("00000002", "00000003", "00000004", "00000005", "00000006"), HoodieTimeline.CLEAN_ACTION));
verifyArchival(getAllArchivedCommitInstants(Collections.emptyList(), HoodieTimeline.CLEAN_ACTION), expectedActiveInstants, commitsAfterArchival);
}
} else {
assertEquals(originalCommits, commitsAfterArchival);
if (!enableMetadata) {
assertEquals(originalCommits, commitsAfterArchival);
} else {
if (i == 7) {
// when i == 7 compaction in metadata table will be triggered and hence archival in datatable will kick in.
// 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", "00000007")));
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);
}
}
}
}
}