cshuo commented on code in PR #18103:
URL: https://github.com/apache/hudi/pull/18103#discussion_r2780260406


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/rebalance/selector/StreamReadBucketIndexKeySelector.java:
##########
@@ -18,14 +18,66 @@
 
 package org.apache.hudi.source.rebalance.selector;
 
+import org.apache.hudi.common.fs.FSUtils;
+import org.apache.hudi.common.util.Option;
+import org.apache.hudi.common.util.collection.Pair;
+import org.apache.hudi.storage.StoragePath;
 import org.apache.hudi.table.format.mor.MergeOnReadInputSplit;
 
 import org.apache.flink.api.java.functions.KeySelector;
 
-public class StreamReadBucketIndexKeySelector implements 
KeySelector<MergeOnReadInputSplit, String> {
+import java.util.List;
+
+public class StreamReadBucketIndexKeySelector implements 
KeySelector<MergeOnReadInputSplit, Pair<String, String>> {
+
+  private final StoragePath tablePath;
+
+  public StreamReadBucketIndexKeySelector(String tablePath) {
+    this.tablePath = new StoragePath(tablePath);
+  }
 
   @Override
-  public String getKey(MergeOnReadInputSplit mergeOnReadInputSplit) throws 
Exception {
-    return mergeOnReadInputSplit.getFileId();
+  public Pair<String, String> getKey(MergeOnReadInputSplit 
mergeOnReadInputSplit) throws Exception {
+    String partitionPath = mergeOnReadInputSplit.getPartitionPath();
+    // handle MergeOnReadInputSplit is restored from state
+    if (partitionPath == null) {
+      Option<String> validFilePath = 
getValidFilePathFromInputSplit(mergeOnReadInputSplit);
+      if (validFilePath.isPresent()) {
+        partitionPath = getPartitionPathFromFullPath(new 
StoragePath(validFilePath.get()), tablePath);

Review Comment:
   add unit tests to ensure the the partition path extracted here is same as 
that from constructor path.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java:
##########
@@ -48,6 +48,7 @@ public class MergeOnReadInputSplit implements InputSplit {
   private final Option<InstantRange> instantRange;
   @Setter
   protected String fileId;
+  private transient String partitionPath;

Review Comment:
   should not be transient?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/cdc/CdcInputSplit.java:
##########


Review Comment:
   Do we still need this constructor?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/format/mor/MergeOnReadInputSplit.java:
##########


Review Comment:
   Do we still need this constructor?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/table/HoodieTableSource.java:
##########
@@ -353,7 +353,9 @@ private HoodieScanContext createHoodieScanContext(RowType 
rowType) {
    */
   private DataStream<MergeOnReadInputSplit> 
addFileDistributionStrategy(SingleOutputStreamOperator<MergeOnReadInputSplit> 
source) {
     if (OptionsResolver.isMorWithBucketIndexUpsert(conf)) {
-      return source.partitionCustom(new 
StreamReadBucketIndexPartitioner(conf.get(FlinkOptions.READ_TASKS)), new 
StreamReadBucketIndexKeySelector());
+      return source.partitionCustom(
+          new StreamReadBucketIndexPartitioner(conf, 
conf.get(FlinkOptions.READ_TASKS)),

Review Comment:
   pass one parameter `conf`only, and get `READ_TASKS` internall.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to