-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-1624] The state based index should bootstrap from existing base… #2581
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,25 +21,41 @@ | |
| import org.apache.hudi.client.FlinkTaskContextSupplier; | ||
| import org.apache.hudi.client.common.HoodieFlinkEngineContext; | ||
| import org.apache.hudi.common.config.SerializableConfiguration; | ||
| import org.apache.hudi.common.fs.FSUtils; | ||
| import org.apache.hudi.common.model.HoodieBaseFile; | ||
| import org.apache.hudi.common.model.HoodieKey; | ||
| import org.apache.hudi.common.model.HoodieRecord; | ||
| import org.apache.hudi.common.model.HoodieRecordLocation; | ||
| import org.apache.hudi.common.model.WriteOperationType; | ||
| import org.apache.hudi.common.util.ParquetUtils; | ||
| import org.apache.hudi.config.HoodieWriteConfig; | ||
| import org.apache.hudi.exception.HoodieException; | ||
| import org.apache.hudi.exception.HoodieIOException; | ||
| import org.apache.hudi.index.HoodieIndexUtils; | ||
| import org.apache.hudi.operator.FlinkOptions; | ||
| import org.apache.hudi.table.HoodieTable; | ||
| import org.apache.hudi.table.action.commit.BucketInfo; | ||
| import org.apache.hudi.util.StreamerUtil; | ||
|
|
||
| import org.apache.flink.annotation.VisibleForTesting; | ||
| import org.apache.flink.api.common.state.MapState; | ||
| import org.apache.flink.api.common.state.MapStateDescriptor; | ||
| import org.apache.flink.api.common.typeinfo.TypeInformation; | ||
| import org.apache.flink.api.common.typeinfo.Types; | ||
| import org.apache.flink.configuration.Configuration; | ||
| import org.apache.flink.runtime.state.CheckpointListener; | ||
| import org.apache.flink.runtime.state.FunctionInitializationContext; | ||
| import org.apache.flink.runtime.state.FunctionSnapshotContext; | ||
| import org.apache.flink.runtime.state.KeyGroupRangeAssignment; | ||
| import org.apache.flink.streaming.api.checkpoint.CheckpointedFunction; | ||
| import org.apache.flink.streaming.api.functions.KeyedProcessFunction; | ||
| import org.apache.flink.util.Collector; | ||
| import org.apache.hadoop.fs.Path; | ||
| import org.slf4j.Logger; | ||
| import org.slf4j.LoggerFactory; | ||
|
|
||
| import java.util.List; | ||
| import java.util.stream.Collectors; | ||
|
|
||
| /** | ||
| * The function to build the write profile incrementally for records within a checkpoint, | ||
|
|
@@ -48,8 +64,8 @@ | |
| * <p>All the records are tagged with HoodieRecordLocation, instead of real instant time, | ||
| * INSERT record uses "I" and UPSERT record uses "U" as instant time. There is no need to keep | ||
| * the "real" instant time for each record, the bucket ID (partition path & fileID) actually decides | ||
| * where the record should write to. The "I" and "U" tag is only used for downstream to decide whether | ||
| * the data bucket is a INSERT or a UPSERT, we should factor the it out when the underneath writer | ||
| * where the record should write to. The "I" and "U" tags are only used for downstream to decide whether | ||
| * the data bucket is an INSERT or an UPSERT, we should factor the tags out when the underneath writer | ||
| * supports specifying the bucket type explicitly. | ||
| * | ||
| * <p>The output records should then shuffle by the bucket ID and thus do scalable write. | ||
|
|
@@ -60,14 +76,51 @@ public class BucketAssignFunction<K, I, O extends HoodieRecord<?>> | |
| extends KeyedProcessFunction<K, I, O> | ||
| implements CheckpointedFunction, CheckpointListener { | ||
|
|
||
| private static final Logger LOG = LoggerFactory.getLogger(BucketAssignFunction.class); | ||
|
|
||
| private HoodieFlinkEngineContext context; | ||
|
|
||
| /** | ||
| * Index cache(speed-up) state for the underneath file based(BloomFilter) indices. | ||
| * When a record came in, we do these check: | ||
| * | ||
| * <ul> | ||
| * <li>Try to load all the records in the partition path where the record belongs to</li> | ||
| * <li>Checks whether the state contains the record key</li> | ||
| * <li>If it does, tag the record with the location</li> | ||
| * <li>If it does not, use the {@link BucketAssigner} to generate a new bucket ID</li> | ||
| * </ul> | ||
| */ | ||
| private MapState<HoodieKey, HoodieRecordLocation> indexState; | ||
|
|
||
| /** | ||
| * Bucket assigner to assign new bucket IDs or reuse existing ones. | ||
| */ | ||
| private BucketAssigner bucketAssigner; | ||
|
|
||
| private final Configuration conf; | ||
|
|
||
| private transient org.apache.hadoop.conf.Configuration hadoopConf; | ||
|
|
||
| private final boolean isChangingRecords; | ||
|
|
||
| /** | ||
| * All the partition paths when the task starts. It is used to help checking whether all the partitions | ||
| * are loaded into the state. | ||
| */ | ||
| private transient List<String> initialPartitionsToLoad; | ||
|
|
||
| /** | ||
| * State to book-keep which partition is loaded into the index state {@code indexState}. | ||
| */ | ||
| private MapState<String, Integer> partitionLoadState; | ||
|
|
||
| /** | ||
| * Whether all partitions are loaded, if it is true, | ||
| * we can only check the state for locations. | ||
| */ | ||
| private boolean allPartitionsLoaded = false; | ||
|
|
||
| public BucketAssignFunction(Configuration conf) { | ||
| this.conf = conf; | ||
| this.isChangingRecords = WriteOperationType.isChangingRecords( | ||
|
|
@@ -78,13 +131,20 @@ public BucketAssignFunction(Configuration conf) { | |
| public void open(Configuration parameters) throws Exception { | ||
| super.open(parameters); | ||
| HoodieWriteConfig writeConfig = StreamerUtil.getHoodieClientConfig(this.conf); | ||
| HoodieFlinkEngineContext context = | ||
| new HoodieFlinkEngineContext( | ||
| new SerializableConfiguration(StreamerUtil.getHadoopConf()), | ||
| new FlinkTaskContextSupplier(getRuntimeContext())); | ||
| this.bucketAssigner = new BucketAssigner( | ||
| context, | ||
| writeConfig); | ||
| this.hadoopConf = StreamerUtil.getHadoopConf(); | ||
| this.context = new HoodieFlinkEngineContext( | ||
| new SerializableConfiguration(this.hadoopConf), | ||
| new FlinkTaskContextSupplier(getRuntimeContext())); | ||
| this.bucketAssigner = new BucketAssigner(context, writeConfig); | ||
| List<String> allPartitionPaths = FSUtils.getAllPartitionPaths(this.context, | ||
| this.conf.getString(FlinkOptions.PATH), false, false, false); | ||
| final int parallelism = getRuntimeContext().getNumberOfParallelSubtasks(); | ||
| final int maxParallelism = getRuntimeContext().getMaxNumberOfParallelSubtasks(); | ||
| final int taskID = getRuntimeContext().getIndexOfThisSubtask(); | ||
| // reference: org.apache.flink.streaming.api.datastream.KeyedStream | ||
| this.initialPartitionsToLoad = allPartitionPaths.stream() | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. when restored from checkpoint,
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Yes welcome to fire a fix and add test cases |
||
| .filter(partition -> KeyGroupRangeAssignment.assignKeyToParallelOperator(partition, maxParallelism, parallelism) == taskID) | ||
| .collect(Collectors.toList()); | ||
| } | ||
|
|
||
| @Override | ||
|
|
@@ -100,6 +160,12 @@ public void initializeState(FunctionInitializationContext context) { | |
| TypeInformation.of(HoodieKey.class), | ||
| TypeInformation.of(HoodieRecordLocation.class)); | ||
| indexState = context.getKeyedStateStore().getMapState(indexStateDesc); | ||
| MapStateDescriptor<String, Integer> partitionLoadStateDesc = | ||
| new MapStateDescriptor<>("partitionLoadState", Types.STRING, Types.INT); | ||
| partitionLoadState = context.getKeyedStateStore().getMapState(partitionLoadStateDesc); | ||
| if (context.isRestored()) { | ||
| checkPartitionsLoaded(); | ||
| } | ||
| } | ||
|
|
||
| @SuppressWarnings("unchecked") | ||
|
|
@@ -112,6 +178,10 @@ public void processElement(I value, Context ctx, Collector<O> out) throws Except | |
| final HoodieKey hoodieKey = record.getKey(); | ||
| final BucketInfo bucketInfo; | ||
| final HoodieRecordLocation location; | ||
| if (!allPartitionsLoaded && !partitionLoadState.contains(hoodieKey.getPartitionPath())) { | ||
|
||
| // If the partition records are never loaded, load the records first. | ||
| loadRecords(hoodieKey.getPartitionPath()); | ||
| } | ||
| // Only changing records need looking up the index for the location, | ||
| // append only records are always recognized as INSERT. | ||
| if (isChangingRecords && this.indexState.contains(hoodieKey)) { | ||
|
|
@@ -146,5 +216,69 @@ public void notifyCheckpointComplete(long l) { | |
| // Refresh the table state when there are new commits. | ||
| this.bucketAssigner.reset(); | ||
| this.bucketAssigner.refreshTable(); | ||
| checkPartitionsLoaded(); | ||
| } | ||
|
|
||
| /** | ||
| * Load all the indices of give partition path into the backup state. | ||
| * | ||
| * @param partitionPath The partition path | ||
| * @throws Exception when error occurs for state update | ||
| */ | ||
| private void loadRecords(String partitionPath) throws Exception { | ||
| HoodieTable<?, ?, ?, ?> hoodieTable = bucketAssigner.getTable(); | ||
| List<HoodieBaseFile> latestBaseFiles = | ||
| HoodieIndexUtils.getLatestBaseFilesForPartition(partitionPath, hoodieTable); | ||
| for (HoodieBaseFile baseFile : latestBaseFiles) { | ||
| List<HoodieKey> hoodieKeys = | ||
| ParquetUtils.fetchRecordKeyPartitionPathFromParquet(hadoopConf, new Path(baseFile.getPath())); | ||
| hoodieKeys.forEach(hoodieKey -> { | ||
| try { | ||
| this.indexState.put(hoodieKey, new HoodieRecordLocation(baseFile.getCommitTime(), baseFile.getFileId())); | ||
| } catch (Exception e) { | ||
| throw new HoodieIOException("Error when load record keys from file: " + baseFile); | ||
| } | ||
| }); | ||
| } | ||
| // Mark the partition path as loaded. | ||
| partitionLoadState.put(partitionPath, 0); | ||
|
||
| } | ||
|
|
||
| /** | ||
| * Checks whether all the partitions of the table are loaded into the state, | ||
| * set the flag {@code allPartitionsLoaded} to true if it is. | ||
| */ | ||
| private void checkPartitionsLoaded() { | ||
| for (String partition : this.initialPartitionsToLoad) { | ||
| try { | ||
| if (!this.partitionLoadState.contains(partition)) { | ||
| return; | ||
| } | ||
| } catch (Exception e) { | ||
| LOG.warn("Error when check whether all partitions are loaded, ignored", e); | ||
| throw new HoodieException(e); | ||
| } | ||
| } | ||
| this.allPartitionsLoaded = true; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| public boolean isAllPartitionsLoaded() { | ||
| return this.allPartitionsLoaded; | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| public void clearIndexState() { | ||
| this.allPartitionsLoaded = false; | ||
| this.indexState.clear(); | ||
| } | ||
|
|
||
| @VisibleForTesting | ||
| public boolean isKeyInState(HoodieKey hoodieKey) { | ||
| try { | ||
| return this.indexState.contains(hoodieKey); | ||
| } catch (Exception e) { | ||
| throw new HoodieException(e); | ||
| } | ||
| } | ||
| } | ||

There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
shall we use this in
getLatestBaseFilesForAllPartitionsto avoid duplicate codes.There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think we could.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It's good that
getLatestBaseFilesForPartitionwas extracted fromgetLatestBaseFilesForAllPartitions.Current codebase:
Maybe the following implementation is more efficient
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
refer to
SparkHoodieBackedTableMetadataWriter#prepRecordsThere was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Agree, there is no need to decide and compare the instant time here, but i would not promote it in this PR, because it is not related.
You can promote it in a separate JIRA issue.