[HUDI-2475] [HUDI-2862] Metadata table creation and avoid bootstrapping race for write client & add locking for upgrade (#4114)
Co-authored-by: Sivabalan Narayanan <n.siva.b@gmail.com>
This commit is contained in:
committed by
GitHub
parent
3a8d64e584
commit
2c7656c35f
@@ -96,21 +96,6 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
public SparkRDDWriteClient(HoodieEngineContext context, HoodieWriteConfig writeConfig,
|
||||
Option<EmbeddedTimelineService> timelineService) {
|
||||
super(context, writeConfig, timelineService);
|
||||
initializeMetadataTable(Option.empty());
|
||||
}
|
||||
|
||||
private void initializeMetadataTable(Option<String> inflightInstantTimestamp) {
|
||||
if (config.isMetadataTableEnabled()) {
|
||||
// Defer bootstrap if upgrade / downgrade is pending
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(
|
||||
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance());
|
||||
if (!upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) {
|
||||
// TODO: Check if we can remove this requirement - auto bootstrap on commit
|
||||
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context, Option.empty(),
|
||||
inflightInstantTimestamp);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
@@ -431,7 +416,8 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
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(), isTableServiceAction));
|
||||
table.getMetadataWriter(hoodieInstant.getTimestamp()).ifPresent(
|
||||
w -> w.update(commitMetadata, hoodieInstant.getTimestamp(), isTableServiceAction));
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -439,37 +425,45 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
|
||||
HoodieTableMetaClient metaClient = createMetaClient(true);
|
||||
UpgradeDowngrade upgradeDowngrade = new UpgradeDowngrade(
|
||||
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance());
|
||||
if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) {
|
||||
if (config.getWriteConcurrencyMode().supportsOptimisticConcurrencyControl()) {
|
||||
this.txnManager.beginTransaction();
|
||||
try {
|
||||
// Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits
|
||||
List<String> instantsToRollback = getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime));
|
||||
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(metaClient);
|
||||
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
|
||||
this.rollbackFailedWrites(pendingRollbacks, true);
|
||||
new UpgradeDowngrade(
|
||||
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance())
|
||||
.run(HoodieTableVersion.current(), instantTime);
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
} else {
|
||||
upgradeDowngrade.run(HoodieTableVersion.current(), instantTime);
|
||||
try {
|
||||
this.txnManager.beginTransaction();
|
||||
if (upgradeDowngrade.needsUpgradeOrDowngrade(HoodieTableVersion.current())) {
|
||||
// Ensure no inflight commits by setting EAGER policy and explicitly cleaning all failed commits
|
||||
List<String> instantsToRollback = getInstantsToRollback(
|
||||
metaClient, HoodieFailedWritesCleaningPolicy.EAGER, Option.of(instantTime));
|
||||
Map<String, Option<HoodiePendingRollbackInfo>> pendingRollbacks = getPendingRollbackInfos(metaClient);
|
||||
instantsToRollback.forEach(entry -> pendingRollbacks.putIfAbsent(entry, Option.empty()));
|
||||
this.rollbackFailedWrites(pendingRollbacks, true);
|
||||
new UpgradeDowngrade(
|
||||
metaClient, config, context, SparkUpgradeDowngradeHelper.getInstance())
|
||||
.run(HoodieTableVersion.current(), instantTime);
|
||||
metaClient.reloadActiveTimeline();
|
||||
initializeMetadataTable(Option.of(instantTime));
|
||||
}
|
||||
metaClient.reloadActiveTimeline();
|
||||
|
||||
// re-bootstrap metadata table if required
|
||||
initializeMetadataTable(Option.of(instantTime));
|
||||
} finally {
|
||||
this.txnManager.endTransaction();
|
||||
}
|
||||
metaClient.validateTableProperties(config.getProps(), operationType);
|
||||
return getTableAndInitCtx(metaClient, operationType, instantTime);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the metadata table if needed. Creating the metadata table writer
|
||||
* will trigger the initial bootstrapping from the data table.
|
||||
*
|
||||
* @param inFlightInstantTimestamp - The in-flight action responsible for the metadata table initialization
|
||||
*/
|
||||
private void initializeMetadataTable(Option<String> inFlightInstantTimestamp) {
|
||||
if (config.isMetadataTableEnabled()) {
|
||||
SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config,
|
||||
context, Option.empty(), inFlightInstantTimestamp);
|
||||
}
|
||||
}
|
||||
|
||||
// TODO : To enforce priority between table service and ingestion writer, use transactions here and invoke strategy
|
||||
private void completeTableService(TableServiceType tableServiceType, HoodieCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses,
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String commitInstant) {
|
||||
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
|
||||
String commitInstant) {
|
||||
|
||||
switch (tableServiceType) {
|
||||
case CLUSTER:
|
||||
|
||||
@@ -122,6 +122,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
}
|
||||
|
||||
protected void commit(HoodieData<HoodieRecord> hoodieDataRecords, String partitionName, String instantTime, boolean canTriggerTableService) {
|
||||
ValidationUtils.checkState(metadataMetaClient != null, "Metadata table is not fully initialized yet.");
|
||||
ValidationUtils.checkState(enabled, "Metadata table cannot be committed to as it is not enabled");
|
||||
JavaRDD<HoodieRecord> records = (JavaRDD<HoodieRecord>) hoodieDataRecords.get();
|
||||
JavaRDD<HoodieRecord> recordRDD = prepRecords(records, partitionName, 1);
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
@@ -38,8 +39,6 @@ import org.apache.hudi.metadata.HoodieTableMetadata;
|
||||
import org.apache.hudi.metadata.HoodieTableMetadataWriter;
|
||||
import org.apache.hudi.metadata.SparkHoodieBackedTableMetadataWriter;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.spark.api.java.JavaRDD;
|
||||
|
||||
import java.io.IOException;
|
||||
@@ -49,8 +48,7 @@ import static org.apache.hudi.data.HoodieJavaRDD.getJavaRDD;
|
||||
public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
extends HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
|
||||
|
||||
private boolean isMetadataAvailabilityUpdated = false;
|
||||
private boolean isMetadataTableAvailable;
|
||||
private volatile boolean isMetadataTableExists = false;
|
||||
|
||||
protected HoodieSparkTable(HoodieWriteConfig config, HoodieEngineContext context, HoodieTableMetaClient metaClient) {
|
||||
super(config, context, metaClient);
|
||||
@@ -112,25 +110,25 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
* @return instance of {@link HoodieTableMetadataWriter}
|
||||
*/
|
||||
@Override
|
||||
public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(Option<T> actionMetadata) {
|
||||
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);
|
||||
public <T extends SpecificRecordBase> Option<HoodieTableMetadataWriter> getMetadataWriter(String triggeringInstantTimestamp,
|
||||
Option<T> actionMetadata) {
|
||||
if (config.isMetadataTableEnabled()) {
|
||||
// Create the metadata table writer. First time after the upgrade this creation might trigger
|
||||
// metadata table bootstrapping. Bootstrapping process could fail and checking the table
|
||||
// existence after the creation is needed.
|
||||
final HoodieTableMetadataWriter metadataWriter = SparkHoodieBackedTableMetadataWriter.create(
|
||||
context.getHadoopConf().get(), config, context, actionMetadata, Option.of(triggeringInstantTimestamp));
|
||||
try {
|
||||
if (isMetadataTableExists || metaClient.getFs().exists(new Path(
|
||||
HoodieTableMetadata.getMetadataTableBasePath(metaClient.getBasePath())))) {
|
||||
isMetadataTableExists = true;
|
||||
return Option.of(metadataWriter);
|
||||
}
|
||||
isMetadataAvailabilityUpdated = true;
|
||||
} catch (IOException e) {
|
||||
throw new HoodieMetadataException("Checking existence of metadata table failed", e);
|
||||
}
|
||||
}
|
||||
if (isMetadataTableAvailable) {
|
||||
return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context,
|
||||
actionMetadata, Option.empty()));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
return Option.empty();
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user