[HUDI-2084] Resend the uncommitted write metadata when start up (#3168)
Co-authored-by: 喻兆靖 <yuzhaojing@bilibili.com>
This commit is contained in:
@@ -58,6 +58,7 @@ import org.apache.hudi.table.MarkerFiles;
|
||||
import org.apache.hudi.table.action.HoodieWriteMetadata;
|
||||
import org.apache.hudi.table.action.compact.FlinkCompactHelpers;
|
||||
import org.apache.hudi.table.upgrade.FlinkUpgradeDowngrade;
|
||||
import org.apache.hudi.util.FlinkClientUtil;
|
||||
|
||||
import com.codahale.metrics.Timer;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@@ -174,7 +175,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
/**
|
||||
* Removes all existing records from the partitions affected and inserts the given HoodieRecords, into the table.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return list of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
@@ -194,7 +195,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
/**
|
||||
* Removes all existing records of the Hoodie table and inserts the given HoodieRecords, into the table.
|
||||
*
|
||||
* @param records HoodieRecords to insert
|
||||
* @param records HoodieRecords to insert
|
||||
* @param instantTime Instant time of the commit
|
||||
* @return list of WriteStatus to inspect errors and counts
|
||||
*/
|
||||
@@ -235,7 +236,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
HoodieTable<T, List<HoodieRecord<T>>, List<HoodieKey>, List<WriteStatus>> table =
|
||||
getTableAndInitCtx(WriteOperationType.DELETE, instantTime);
|
||||
preWrite(instantTime, WriteOperationType.DELETE, table.getMetaClient());
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = table.delete(context,instantTime, keys);
|
||||
HoodieWriteMetadata<List<WriteStatus>> result = table.delete(context, instantTime, keys);
|
||||
return postWrite(result, instantTime, table);
|
||||
}
|
||||
|
||||
@@ -391,11 +392,11 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
/**
|
||||
* Get or create a new write handle in order to reuse the file handles.
|
||||
*
|
||||
* @param record The first record in the bucket
|
||||
* @param config Write config
|
||||
* @param instantTime The instant time
|
||||
* @param table The table
|
||||
* @param recordItr Record iterator
|
||||
* @param record The first record in the bucket
|
||||
* @param config Write config
|
||||
* @param instantTime The instant time
|
||||
* @param table The table
|
||||
* @param recordItr Record iterator
|
||||
* @return Existing write handle or create a new one
|
||||
*/
|
||||
private HoodieWriteHandle<?, ?, ?, ?> getOrCreateWriteHandle(
|
||||
@@ -454,7 +455,8 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
public String getLastPendingInstant(String actionType) {
|
||||
HoodieTimeline unCompletedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterInflightsAndRequested();
|
||||
HoodieTimeline unCompletedTimeline = FlinkClientUtil.createMetaClient(basePath)
|
||||
.getCommitsTimeline().filterInflightsAndRequested();
|
||||
return unCompletedTimeline.getInstants()
|
||||
.filter(x -> x.getAction().equals(actionType))
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
@@ -465,7 +467,8 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
|
||||
public String getLastCompletedInstant(HoodieTableType tableType) {
|
||||
final String commitType = CommitUtils.getCommitActionType(tableType);
|
||||
HoodieTimeline completedTimeline = getHoodieTable().getMetaClient().getCommitsTimeline().filterCompletedInstants();
|
||||
HoodieTimeline completedTimeline = FlinkClientUtil.createMetaClient(basePath)
|
||||
.getCommitsTimeline().filterCompletedInstants();
|
||||
return completedTimeline.getInstants()
|
||||
.filter(x -> x.getAction().equals(commitType))
|
||||
.map(HoodieInstant::getTimestamp)
|
||||
@@ -475,8 +478,7 @@ public class HoodieFlinkWriteClient<T extends HoodieRecordPayload> extends
|
||||
}
|
||||
|
||||
public void transitionRequestedToInflight(String commitType, String inFlightInstant) {
|
||||
HoodieFlinkTable<T> table = getHoodieTable();
|
||||
HoodieActiveTimeline activeTimeline = table.getActiveTimeline();
|
||||
HoodieActiveTimeline activeTimeline = FlinkClientUtil.createMetaClient(basePath).getActiveTimeline();
|
||||
HoodieInstant requested = new HoodieInstant(HoodieInstant.State.REQUESTED, commitType, inFlightInstant);
|
||||
activeTimeline.transitionRequestedToInflight(requested, Option.empty(),
|
||||
config.shouldAllowMultiWriteOnSameInstant());
|
||||
|
||||
@@ -18,6 +18,8 @@
|
||||
|
||||
package org.apache.hudi.util;
|
||||
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient;
|
||||
|
||||
import org.apache.flink.api.java.hadoop.mapred.utils.HadoopUtils;
|
||||
import org.apache.flink.configuration.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
@@ -29,6 +31,13 @@ import java.io.File;
|
||||
*/
|
||||
public class FlinkClientUtil {
|
||||
|
||||
/**
|
||||
* Creates the meta client.
|
||||
*/
|
||||
public static HoodieTableMetaClient createMetaClient(String basePath) {
|
||||
return HoodieTableMetaClient.builder().setBasePath(basePath).setConf(FlinkClientUtil.getHadoopConf()).build();
|
||||
}
|
||||
|
||||
/**
|
||||
* Parses the file name from path.
|
||||
*/
|
||||
|
||||
Reference in New Issue
Block a user