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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/assign/DefaultHoodieSplitAssigner.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.source.assign;
+
+import org.apache.hudi.source.split.HoodieSourceSplit;
+
+import java.io.Serializable;
+
+/**
+ * Default implementation of {@link HoodieSplitAssigner} that assigns Hoodie
+ * source splits to task IDs using round-robin distribution based on split 
number.
+ */
+public class DefaultHoodieSplitAssigner implements HoodieSplitAssigner, 
Serializable {

Review Comment:
   Seems the split assigner doesn't need to be Serializable, it's a field of 
`DefaultHoodieSplitProvider`, which is not serializable. 



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/assign/HoodieSplitAssigner.java:
##########
@@ -0,0 +1,35 @@
+/*
+ * 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.source.assign;
+
+import org.apache.hudi.source.split.HoodieSourceSplit;
+
+/**
+ * Interface for assigning {@link HoodieSourceSplit} instances to task IDs.
+ */
+public interface HoodieSplitAssigner {

Review Comment:
   For legacy source, there are 3 kinds split distribution strategy:
   1) distribute by fileId ;
   2) distribute by bucketId;
   3) distribute by split number;
   
   Seems 1 is missed in this PR.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/split/DefaultHoodieSplitProvider.java:
##########
@@ -37,29 +40,45 @@ public class DefaultHoodieSplitProvider implements 
HoodieSplitProvider {
   public static final int DEFAULT_SPLIT_QUEUE_SIZE = 20;
 
   private Queue<HoodieSourceSplit> pendingSplits;
+  private HoodieSplitAssigner splitAssigner;
 
-  public DefaultHoodieSplitProvider() {
+  public DefaultHoodieSplitProvider(HoodieSplitAssigner splitAssigner) {
     this.pendingSplits = new ConcurrentLinkedDeque<>();
+    this.splitAssigner = splitAssigner;
   }
 
   /**
    * Creates a DefaultHoodieSplitProvider with a custom comparator for 
ordering splits.
    *
-   * @param comparator the comparator to use for ordering splits
+   * @param comparator the comparator to use for ordering splits (must not be 
null)
+   * @param splitAssigner the assigner that assigns each split to a task ID 
(must not be null)
    */
-  public DefaultHoodieSplitProvider(SerializableComparator<HoodieSourceSplit> 
comparator) {
-    ValidationUtils.checkArgument(comparator != null,
-        "The hoodie source split comparator can't be null");
+  public DefaultHoodieSplitProvider(
+          SerializableComparator<HoodieSourceSplit> comparator,
+          HoodieSplitAssigner splitAssigner) {
     this.pendingSplits = new PriorityBlockingQueue<>(DEFAULT_SPLIT_QUEUE_SIZE, 
comparator);
+    this.splitAssigner = splitAssigner;
   }
 
   @Override
-  public Option<HoodieSourceSplit> getNext(@Nullable String hostname) {
-    if (!pendingSplits.isEmpty()) {
-      return Option.of(pendingSplits.poll());
+  public Option<HoodieSourceSplit> getNext(int subTaskId, @Nullable String 
hostname) {
+    List<HoodieSourceSplit> visited = new ArrayList<>();
+    HoodieSourceSplit targetSplit = null;
+    while (!pendingSplits.isEmpty()) {
+      HoodieSourceSplit split = pendingSplits.poll();
+      int target = splitAssigner.assign(split);
+      if (subTaskId != target) {
+        visited.add(split);
+      } else {
+        targetSplit = split;
+        break;
+      }
     }
 
-    return Option.empty();
+    if (!visited.isEmpty()) {
+      pendingSplits.addAll(visited);

Review Comment:
   Maybe we can use structure like `Map<Integer,  Queue<HoodieSourceSplit>>` to 
keep mapping: subtask -> pending splits. 
   
   Splits are assigned during addition, and can be returned directly when 
requested.



-- 
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