1
0

[HUDI-845] Added locking capability to allow multiple writers (#2374)

* [HUDI-845] Added locking capability to allow multiple writers
1. Added LockProvider API for pluggable lock methodologies
2. Added Resolution Strategy API to allow for pluggable conflict resolution
3. Added TableService client API to schedule table services
4. Added Transaction Manager for wrapping actions within transactions
This commit is contained in:
n3nash
2021-03-16 16:43:53 -07:00
committed by GitHub
parent b038623ed3
commit 74241947c1
88 changed files with 4876 additions and 381 deletions

View File

@@ -22,15 +22,18 @@ import com.codahale.metrics.Timer;
import org.apache.hadoop.conf.Configuration;
import org.apache.hudi.client.common.HoodieSparkEngineContext;
import org.apache.hudi.client.embedded.EmbeddedTimelineService;
import org.apache.hudi.client.utils.TransactionUtils;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.fs.HoodieWrapperFileSystem;
import org.apache.hudi.common.metrics.Registry;
import org.apache.hudi.common.model.HoodieCommitMetadata;
import org.apache.hudi.common.model.HoodieFailedWritesCleaningPolicy;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.model.HoodieReplaceCommitMetadata;
import org.apache.hudi.common.model.HoodieWriteStat;
import org.apache.hudi.common.model.TableServiceType;
import org.apache.hudi.common.model.WriteOperationType;
import org.apache.hudi.common.table.HoodieTableMetaClient;
import org.apache.hudi.common.table.HoodieTableVersion;
@@ -52,6 +55,7 @@ import org.apache.hudi.table.HoodieSparkTable;
import org.apache.hudi.table.HoodieTable;
import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.compact.SparkCompactHelpers;
import org.apache.hudi.table.upgrade.AbstractUpgradeDowngrade;
import org.apache.hudi.table.upgrade.SparkUpgradeDowngrade;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
@@ -146,7 +150,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.UPSERT, instantTime);
table.validateUpsertSchema();
preWrite(instantTime, WriteOperationType.UPSERT);
preWrite(instantTime, WriteOperationType.UPSERT, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsert(context, instantTime, records);
if (result.getIndexLookupDuration().isPresent()) {
metrics.updateIndexMetrics(LOOKUP_STR, result.getIndexLookupDuration().get().toMillis());
@@ -159,7 +163,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.UPSERT_PREPPED, instantTime);
table.validateUpsertSchema();
preWrite(instantTime, WriteOperationType.UPSERT_PREPPED);
preWrite(instantTime, WriteOperationType.UPSERT_PREPPED, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.upsertPrepped(context,instantTime, preppedRecords);
return postWrite(result, instantTime, table);
}
@@ -169,7 +173,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.INSERT, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT);
preWrite(instantTime, WriteOperationType.INSERT, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insert(context,instantTime, records);
return postWrite(result, instantTime, table);
}
@@ -179,7 +183,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.INSERT_PREPPED, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT_PREPPED);
preWrite(instantTime, WriteOperationType.INSERT_PREPPED, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.insertPrepped(context,instantTime, preppedRecords);
return postWrite(result, instantTime, table);
}
@@ -194,7 +198,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
public HoodieWriteResult insertOverwrite(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE);
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE, table.getMetaClient());
HoodieWriteMetadata result = table.insertOverwrite(context, instantTime, records);
return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
}
@@ -210,7 +214,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
public HoodieWriteResult insertOverwriteTable(JavaRDD<HoodieRecord<T>> records, final String instantTime) {
HoodieTable table = getTableAndInitCtx(WriteOperationType.INSERT_OVERWRITE_TABLE, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE);
preWrite(instantTime, WriteOperationType.INSERT_OVERWRITE_TABLE, table.getMetaClient());
HoodieWriteMetadata result = table.insertOverwriteTable(context, instantTime, records);
return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
}
@@ -225,7 +229,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.BULK_INSERT, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.BULK_INSERT);
preWrite(instantTime, WriteOperationType.BULK_INSERT, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsert(context,instantTime, records, userDefinedBulkInsertPartitioner);
return postWrite(result, instantTime, table);
}
@@ -235,7 +239,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table =
getTableAndInitCtx(WriteOperationType.BULK_INSERT_PREPPED, instantTime);
table.validateInsertSchema();
preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED);
preWrite(instantTime, WriteOperationType.BULK_INSERT_PREPPED, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.bulkInsertPrepped(context,instantTime, preppedRecords, bulkInsertPartitioner);
return postWrite(result, instantTime, table);
}
@@ -243,14 +247,14 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
public JavaRDD<WriteStatus> delete(JavaRDD<HoodieKey> keys, String instantTime) {
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table = getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
preWrite(instantTime, WriteOperationType.DELETE);
preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.delete(context,instantTime, keys);
return postWrite(result, instantTime, table);
}
public HoodieWriteResult deletePartitions(List<String> partitions, String instantTime) {
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table = getTableAndInitCtx(WriteOperationType.DELETE_PARTITION, instantTime);
preWrite(instantTime, WriteOperationType.DELETE_PARTITION);
preWrite(instantTime, WriteOperationType.DELETE_PARTITION, table.getMetaClient());
HoodieWriteMetadata<JavaRDD<WriteStatus>> result = table.deletePartitions(context,instantTime, partitions);
return new HoodieWriteResult(postWrite(result, instantTime, table), result.getPartitionToReplaceFileIds());
}
@@ -311,6 +315,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
protected JavaRDD<WriteStatus> compact(String compactionInstantTime, boolean shouldComplete) {
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
preWrite(compactionInstantTime, WriteOperationType.COMPACT, table.getMetaClient());
HoodieTimeline pendingCompactionTimeline = table.getActiveTimeline().filterPendingCompactionTimeline();
HoodieInstant inflightInstant = HoodieTimeline.getCompactionInflightInstant(compactionInstantTime);
if (pendingCompactionTimeline.containsInstant(inflightInstant)) {
@@ -321,7 +326,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
HoodieWriteMetadata<JavaRDD<WriteStatus>> compactionMetadata = table.compact(context, compactionInstantTime);
JavaRDD<WriteStatus> statuses = compactionMetadata.getWriteStatuses();
if (shouldComplete && compactionMetadata.getCommitMetadata().isPresent()) {
completeCompaction(compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
completeTableService(TableServiceType.COMPACT, compactionMetadata.getCommitMetadata().get(), statuses, table, compactionInstantTime);
}
return statuses;
}
@@ -329,6 +334,7 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
public HoodieWriteMetadata<JavaRDD<WriteStatus>> cluster(String clusteringInstant, boolean shouldComplete) {
HoodieSparkTable<T> table = HoodieSparkTable.create(config, context);
preWrite(clusteringInstant, WriteOperationType.CLUSTER, table.getMetaClient());
HoodieTimeline pendingClusteringTimeline = table.getActiveTimeline().filterPendingReplaceTimeline();
HoodieInstant inflightInstant = HoodieTimeline.getReplaceCommitInflightInstant(clusteringInstant);
if (pendingClusteringTimeline.containsInstant(inflightInstant)) {
@@ -339,13 +345,14 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
LOG.info("Starting clustering at " + clusteringInstant);
HoodieWriteMetadata<JavaRDD<WriteStatus>> clusteringMetadata = table.cluster(context, clusteringInstant);
JavaRDD<WriteStatus> statuses = clusteringMetadata.getWriteStatuses();
// TODO : Where is shouldComplete used ?
if (shouldComplete && clusteringMetadata.getCommitMetadata().isPresent()) {
completeClustering((HoodieReplaceCommitMetadata) clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant);
completeTableService(TableServiceType.CLUSTER, clusteringMetadata.getCommitMetadata().get(), statuses, table, clusteringInstant);
}
return clusteringMetadata;
}
protected void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses,
private void completeClustering(HoodieReplaceCommitMetadata metadata, JavaRDD<WriteStatus> writeStatuses,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String clusteringCommitTime) {
@@ -380,11 +387,44 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
@Override
protected HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(WriteOperationType operationType, String instantTime) {
HoodieTableMetaClient metaClient = createMetaClient(true);
new SparkUpgradeDowngrade(metaClient, config, context).run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
return getTableAndInitCtx(metaClient, operationType);
AbstractUpgradeDowngrade upgradeDowngrade = new SparkUpgradeDowngrade(metaClient, config, context);
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
this.rollbackFailedWrites(getInstantsToRollback(metaClient, HoodieFailedWritesCleaningPolicy.EAGER));
new SparkUpgradeDowngrade(metaClient, config, context)
.run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
} finally {
this.txnManager.endTransaction();
}
} else {
upgradeDowngrade.run(metaClient, HoodieTableVersion.current(), config, context, instantTime);
}
}
return getTableAndInitCtx(metaClient, operationType, instantTime);
}
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(HoodieTableMetaClient metaClient, WriteOperationType operationType) {
// 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) {
switch (tableServiceType) {
case CLUSTER:
completeClustering((HoodieReplaceCommitMetadata) metadata, writeStatuses, table, commitInstant);
break;
case COMPACT:
completeCompaction(metadata, writeStatuses, table, commitInstant);
break;
default:
throw new IllegalArgumentException("This table service is not valid " + tableServiceType);
}
}
private HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> getTableAndInitCtx(
HoodieTableMetaClient metaClient, WriteOperationType operationType, String instantTime) {
if (operationType == WriteOperationType.DELETE) {
setWriteSchemaForDeletes(metaClient);
}
@@ -408,6 +448,15 @@ public class SparkRDDWriteClient<T extends HoodieRecordPayload> extends
}
}
@Override
protected void preCommit(String instantTime, 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());
}
@Override
protected void initWrapperFSMetrics() {
if (config.isMetricsOn()) {

View File

@@ -19,6 +19,7 @@
package org.apache.hudi.table;
import org.apache.hudi.avro.model.HoodieCleanMetadata;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.avro.model.HoodieClusteringPlan;
import org.apache.hudi.avro.model.HoodieCompactionPlan;
import org.apache.hudi.avro.model.HoodieRestoreMetadata;
@@ -45,8 +46,9 @@ import org.apache.hudi.table.action.HoodieWriteMetadata;
import org.apache.hudi.table.action.bootstrap.HoodieBootstrapWriteMetadata;
import org.apache.hudi.table.action.bootstrap.SparkBootstrapCommitActionExecutor;
import org.apache.hudi.table.action.clean.SparkCleanActionExecutor;
import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor;
import org.apache.hudi.table.action.clean.SparkCleanPlanActionExecutor;
import org.apache.hudi.table.action.cluster.SparkClusteringPlanActionExecutor;
import org.apache.hudi.table.action.cluster.SparkExecuteClusteringCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkBulkInsertCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkBulkInsertPreppedCommitActionExecutor;
import org.apache.hudi.table.action.commit.SparkDeleteCommitActionExecutor;
@@ -176,6 +178,11 @@ public class HoodieSparkCopyOnWriteTable<T extends HoodieRecordPayload> extends
new SparkCopyOnWriteRestoreActionExecutor((HoodieSparkEngineContext) context, config, this, instantTime, HoodieTimeline.INIT_INSTANT_TS).execute();
}
@Override
public Option<HoodieCleanerPlan> scheduleCleaning(HoodieEngineContext context, String instantTime, Option<Map<String, String>> extraMetadata) {
return new SparkCleanPlanActionExecutor<>(context, config,this, instantTime, extraMetadata).execute();
}
public Iterator<List<WriteStatus>> handleUpdate(String instantTime, String partitionPath, String fileId,
Map<String, HoodieRecord<T>> keyToNewRecords, HoodieBaseFile oldDataFile) throws IOException {
// these are updates

View File

@@ -57,9 +57,12 @@ import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.exception.HoodieCommitException;
import org.apache.hudi.exception.HoodieException;
import org.apache.hudi.exception.HoodieIOException;
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.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;
@@ -222,6 +225,17 @@ 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);
@@ -237,7 +251,7 @@ 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);

View File

@@ -0,0 +1,55 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hudi.table.action.clean;
import org.apache.hudi.avro.model.HoodieCleanerPlan;
import org.apache.hudi.client.WriteStatus;
import org.apache.hudi.common.engine.HoodieEngineContext;
import org.apache.hudi.common.model.HoodieKey;
import org.apache.hudi.common.model.HoodieRecord;
import org.apache.hudi.common.model.HoodieRecordPayload;
import org.apache.hudi.common.util.Option;
import org.apache.hudi.config.HoodieWriteConfig;
import org.apache.hudi.table.HoodieTable;
import org.apache.log4j.LogManager;
import org.apache.log4j.Logger;
import org.apache.spark.api.java.JavaRDD;
import java.util.Map;
@SuppressWarnings("checkstyle:LineLength")
public class SparkCleanPlanActionExecutor<T extends HoodieRecordPayload> extends
BaseCleanPlanActionExecutor<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> {
private static final Logger LOG = LogManager.getLogger(SparkCleanPlanActionExecutor.class);
public SparkCleanPlanActionExecutor(HoodieEngineContext context,
HoodieWriteConfig config,
HoodieTable<T, JavaRDD<HoodieRecord<T>>, JavaRDD<HoodieKey>, JavaRDD<WriteStatus>> table,
String instantTime,
Option<Map<String, String>> extraMetadata) {
super(context, config, table, instantTime, extraMetadata);
}
@Override
protected Option<HoodieCleanerPlan> createCleanerPlan() {
return super.execute();
}
}

View File

@@ -38,12 +38,15 @@ import org.apache.hudi.common.util.ReflectionUtils;
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.HoodieMetadataException;
import org.apache.hudi.exception.HoodieUpsertException;
import org.apache.hudi.execution.SparkLazyInsertIterable;
import org.apache.hudi.io.CreateHandleFactory;
import org.apache.hudi.io.HoodieMergeHandle;
import org.apache.hudi.io.HoodieSortedMergeHandle;
import org.apache.hudi.io.storage.HoodieConcatHandle;
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;
@@ -244,7 +247,7 @@ 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();
@@ -340,6 +343,17 @@ public abstract class BaseSparkCommitActionExecutor<T extends HoodieRecordPayloa
partitionPath, fileId, dataFileToBeMerged, taskContextSupplier);
}
@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 {