[HUDI-2468] Metadata table support for rolling back the first commit (#3843)
- Fix is to make Metadata table writer creation aware of the currently inflight action so that it can make some informed decision about whether bootstrapping is needed for the table and whether any pending action on the data timeline can be ignored.
This commit is contained in:
committed by
GitHub
parent
5ed35bff83
commit
c9d641cc30
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.metadata;
|
||||
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
import org.apache.hudi.client.SparkRDDWriteClient;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
@@ -47,12 +48,23 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
|
||||
private static final Logger LOG = LogManager.getLogger(SparkHoodieBackedTableMetadataWriter.class);
|
||||
|
||||
public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig, HoodieEngineContext context) {
|
||||
return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context);
|
||||
public static HoodieTableMetadataWriter create(Configuration conf, HoodieWriteConfig writeConfig,
|
||||
HoodieEngineContext context) {
|
||||
return create(conf, writeConfig, context, Option.empty());
|
||||
}
|
||||
|
||||
SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf, HoodieWriteConfig writeConfig, HoodieEngineContext engineContext) {
|
||||
super(hadoopConf, writeConfig, engineContext);
|
||||
public static <T extends SpecificRecordBase> HoodieTableMetadataWriter create(Configuration conf,
|
||||
HoodieWriteConfig writeConfig,
|
||||
HoodieEngineContext context,
|
||||
Option<T> actionMetadata) {
|
||||
return new SparkHoodieBackedTableMetadataWriter(conf, writeConfig, context, actionMetadata);
|
||||
}
|
||||
|
||||
<T extends SpecificRecordBase> SparkHoodieBackedTableMetadataWriter(Configuration hadoopConf,
|
||||
HoodieWriteConfig writeConfig,
|
||||
HoodieEngineContext engineContext,
|
||||
Option<T> actionMetadata) {
|
||||
super(hadoopConf, writeConfig, engineContext, actionMetadata);
|
||||
}
|
||||
|
||||
@Override
|
||||
@@ -71,7 +83,8 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void initialize(HoodieEngineContext engineContext) {
|
||||
protected <T extends SpecificRecordBase> void initialize(HoodieEngineContext engineContext,
|
||||
Option<T> actionMetadata) {
|
||||
try {
|
||||
metrics.map(HoodieMetadataMetrics::registry).ifPresent(registry -> {
|
||||
if (registry instanceof DistributedRegistry) {
|
||||
@@ -81,7 +94,7 @@ public class SparkHoodieBackedTableMetadataWriter extends HoodieBackedTableMetad
|
||||
});
|
||||
|
||||
if (enabled) {
|
||||
bootstrapIfNeeded(engineContext, dataMetaClient);
|
||||
bootstrapIfNeeded(engineContext, dataMetaClient, actionMetadata);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Failed to initialize metadata table. Disabling the writer.", e);
|
||||
|
||||
@@ -18,6 +18,7 @@
|
||||
|
||||
package org.apache.hudi.table;
|
||||
|
||||
import org.apache.avro.specific.SpecificRecordBase;
|
||||
import org.apache.hudi.client.WriteStatus;
|
||||
import org.apache.hudi.client.common.HoodieSparkEngineContext;
|
||||
import org.apache.hudi.common.data.HoodieData;
|
||||
@@ -111,11 +112,11 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
* @return instance of {@link HoodieTableMetadataWriter}
|
||||
*/
|
||||
@Override
|
||||
public Option<HoodieTableMetadataWriter> getMetadataWriter() {
|
||||
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().
|
||||
// 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())));
|
||||
@@ -126,7 +127,8 @@ public abstract class HoodieSparkTable<T extends HoodieRecordPayload>
|
||||
}
|
||||
}
|
||||
if (isMetadataTableAvailable) {
|
||||
return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context));
|
||||
return Option.of(SparkHoodieBackedTableMetadataWriter.create(context.getHadoopConf().get(), config, context,
|
||||
actionMetadata));
|
||||
} else {
|
||||
return Option.empty();
|
||||
}
|
||||
|
||||
@@ -337,7 +337,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
public void testSimpleTagLocationAndUpdateWithRollback() throws Exception {
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfigBuilder(100, false, false)
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
@@ -425,7 +425,7 @@ public class TestHBaseIndex extends SparkClientFunctionalTestHarness {
|
||||
public void testEnsureTagLocationUsesCommitTimeline() throws Exception {
|
||||
// Load to memory
|
||||
HoodieWriteConfig config = getConfigBuilder(100, false, false)
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(false).build()).build();
|
||||
.withMetadataConfig(HoodieMetadataConfig.newBuilder().enable(true).build()).build();
|
||||
SparkHoodieHBaseIndex index = new SparkHoodieHBaseIndex(config);
|
||||
SparkRDDWriteClient writeClient = getHoodieWriteClient(config);
|
||||
|
||||
|
||||
@@ -459,6 +459,39 @@ public class TestHoodieBackedMetadata extends TestHoodieMetadataBase {
|
||||
|
||||
// Some operations are not feasible with test table infra. hence using write client to test those cases.
|
||||
|
||||
/**
|
||||
* Rollback of the first commit should not trigger bootstrap errors at the metadata table.
|
||||
*/
|
||||
@ParameterizedTest
|
||||
@EnumSource(HoodieTableType.class)
|
||||
public void testFirstCommitRollback(HoodieTableType tableType) throws Exception {
|
||||
init(tableType);
|
||||
HoodieSparkEngineContext engineContext = new HoodieSparkEngineContext(jsc);
|
||||
|
||||
try (SparkRDDWriteClient client = new SparkRDDWriteClient(engineContext, getWriteConfig(true, true))) {
|
||||
|
||||
// Write 1
|
||||
String commitTime = "0000001";
|
||||
List<HoodieRecord> records = dataGen.generateInserts(commitTime, 20);
|
||||
client.startCommitWithTime(commitTime);
|
||||
List<WriteStatus> writeStatuses = client.insert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
|
||||
// Rollback the first commit
|
||||
client.rollback(commitTime);
|
||||
|
||||
// Write 2
|
||||
commitTime = "0000002";
|
||||
records = dataGen.generateInserts(commitTime, 10);
|
||||
client.startCommitWithTime(commitTime);
|
||||
writeStatuses = client.upsert(jsc.parallelize(records, 1), commitTime).collect();
|
||||
assertNoWriteErrors(writeStatuses);
|
||||
validateMetadata(client);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
/**
|
||||
* Test several table operations with restore. This test uses SparkRDDWriteClient.
|
||||
* Once the restore support is ready in HoodieTestTable, then rewrite this test.
|
||||
|
||||
Reference in New Issue
Block a user