xushiyan commented on code in PR #17562:
URL: https://github.com/apache/hudi/pull/17562#discussion_r2615926103


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.enumerator;
+
+import org.apache.hudi.source.ScanContext;
+import org.apache.hudi.source.split.HoodieContinuousSplitBatch;
+import org.apache.hudi.source.split.HoodieContinuousSplitDiscover;
+import org.apache.hudi.source.split.HoodieSourceSplit;
+import org.apache.hudi.source.split.HoodieSplitProvider;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi 
commits of upstream Hoodie table.
+ */
+public class HoodieContinuousSplitEnumerator extends 
AbstractHoodieSplitEnumerator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class);
+
+  private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext;
+  private final HoodieSplitProvider splitProvider;
+  private final HoodieContinuousSplitDiscover splitDiscover;
+  private final ScanContext scanContext;
+
+  /**
+   * Instant for the last enumerated commit. Next incremental enumeration 
should be based off
+   * this as the starting position.
+   */
+  private final AtomicReference<HoodieEnumeratorPosition> position;
+
+  public HoodieContinuousSplitEnumerator(
+      SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext,
+      HoodieSplitProvider splitProvider,
+      HoodieContinuousSplitDiscover splitDiscover,
+      ScanContext scanContext,
+      @Nullable HoodieSplitEnumeratorState enumState) {

Review Comment:
   let's try to avoid the case of passing null, using `Option<>` instead 
(Hudi's own `org.apache.hudi.common.util.Option`)



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.enumerator;
+
+import org.apache.hudi.source.ScanContext;
+import org.apache.hudi.source.split.HoodieContinuousSplitBatch;
+import org.apache.hudi.source.split.HoodieContinuousSplitDiscover;
+import org.apache.hudi.source.split.HoodieSourceSplit;
+import org.apache.hudi.source.split.HoodieSplitProvider;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi 
commits of upstream Hoodie table.
+ */
+public class HoodieContinuousSplitEnumerator extends 
AbstractHoodieSplitEnumerator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class);
+
+  private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext;
+  private final HoodieSplitProvider splitProvider;
+  private final HoodieContinuousSplitDiscover splitDiscover;
+  private final ScanContext scanContext;
+
+  /**
+   * Instant for the last enumerated commit. Next incremental enumeration 
should be based off
+   * this as the starting position.
+   */
+  private final AtomicReference<HoodieEnumeratorPosition> position;
+
+  public HoodieContinuousSplitEnumerator(
+      SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext,
+      HoodieSplitProvider splitProvider,
+      HoodieContinuousSplitDiscover splitDiscover,
+      ScanContext scanContext,
+      @Nullable HoodieSplitEnumeratorState enumState) {
+    super(enumeratorContext, splitProvider);
+    this.enumeratorContext = enumeratorContext;
+    this.splitProvider = splitProvider;
+    this.splitDiscover = splitDiscover;
+    this.scanContext = scanContext;
+    this.position = new AtomicReference<>();
+
+    if (enumState != null) {
+      
this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(),
 enumState.getLastEnumeratedInstantCompletionTime()));
+    } else {
+      
this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), 
scanContext.getStartInstant()));
+    }
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    enumeratorContext.callAsync(
+        this::discoverSplits,
+        this::processDiscoveredSplits,
+        0L,
+        scanContext.getScanInterval().toMillis());
+  }
+
+  @Override
+  boolean shouldWaitForMoreSplits() {
+    return true;
+  }
+
+  @Override
+  public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws 
Exception {
+    return new HoodieSplitEnumeratorState(splitProvider.state(), 
position.get().lastInstant(), position.get().lastInstantCompletionTime());
+  }
+
+  private HoodieContinuousSplitBatch discoverSplits() {
+    int pendingSplitNumber = splitProvider.pendingSplitCount();
+    if (pendingSplitNumber > scanContext.getMaxPendingSplits()) {
+      LOG.info(
+          "Pause split discovery as the assigner already has too many pending 
splits: {}",
+          pendingSplitNumber);
+      return HoodieContinuousSplitBatch.EMPTY;
+    }
+    return 
splitDiscover.discoverSplits(position.get().lastInstantCompletionTime());
+  }
+
+  private void processDiscoveredSplits(HoodieContinuousSplitBatch result, 
Throwable throwable) {
+    if (throwable == null) {
+      if (!Objects.equals(result.getFromInstant(), 
position.get().lastInstantCompletionTime())) {

Review Comment:
   prefer not to read negation of condition check
   ```suggestion
         if (Objects.equals(result.getFromInstant(), 
position.get().lastInstantCompletionTime())) {
   ```



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.enumerator;
+
+import org.apache.hudi.source.ScanContext;
+import org.apache.hudi.source.split.HoodieContinuousSplitBatch;
+import org.apache.hudi.source.split.HoodieContinuousSplitDiscover;
+import org.apache.hudi.source.split.HoodieSourceSplit;
+import org.apache.hudi.source.split.HoodieSplitProvider;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi 
commits of upstream Hoodie table.
+ */
+public class HoodieContinuousSplitEnumerator extends 
AbstractHoodieSplitEnumerator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class);
+
+  private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext;
+  private final HoodieSplitProvider splitProvider;
+  private final HoodieContinuousSplitDiscover splitDiscover;
+  private final ScanContext scanContext;
+
+  /**
+   * Instant for the last enumerated commit. Next incremental enumeration 
should be based off
+   * this as the starting position.
+   */
+  private final AtomicReference<HoodieEnumeratorPosition> position;
+
+  public HoodieContinuousSplitEnumerator(
+      SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext,
+      HoodieSplitProvider splitProvider,
+      HoodieContinuousSplitDiscover splitDiscover,
+      ScanContext scanContext,
+      @Nullable HoodieSplitEnumeratorState enumState) {
+    super(enumeratorContext, splitProvider);
+    this.enumeratorContext = enumeratorContext;
+    this.splitProvider = splitProvider;
+    this.splitDiscover = splitDiscover;
+    this.scanContext = scanContext;
+    this.position = new AtomicReference<>();
+
+    if (enumState != null) {
+      
this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(),
 enumState.getLastEnumeratedInstantCompletionTime()));
+    } else {
+      
this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), 
scanContext.getStartInstant()));
+    }
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    enumeratorContext.callAsync(
+        this::discoverSplits,
+        this::processDiscoveredSplits,
+        0L,
+        scanContext.getScanInterval().toMillis());
+  }
+
+  @Override
+  boolean shouldWaitForMoreSplits() {
+    return true;
+  }
+
+  @Override
+  public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws 
Exception {
+    return new HoodieSplitEnumeratorState(splitProvider.state(), 
position.get().lastInstant(), position.get().lastInstantCompletionTime());
+  }
+
+  private HoodieContinuousSplitBatch discoverSplits() {
+    int pendingSplitNumber = splitProvider.pendingSplitCount();
+    if (pendingSplitNumber > scanContext.getMaxPendingSplits()) {
+      LOG.info(
+          "Pause split discovery as the assigner already has too many pending 
splits: {}",
+          pendingSplitNumber);
+      return HoodieContinuousSplitBatch.EMPTY;
+    }
+    return 
splitDiscover.discoverSplits(position.get().lastInstantCompletionTime());
+  }
+
+  private void processDiscoveredSplits(HoodieContinuousSplitBatch result, 
Throwable throwable) {
+    if (throwable == null) {
+      if (!Objects.equals(result.getFromInstant(), 
position.get().lastInstantCompletionTime())) {

Review Comment:
   then update the if else block according



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.enumerator;
+
+import org.apache.hudi.source.ScanContext;
+import org.apache.hudi.source.split.HoodieContinuousSplitBatch;
+import org.apache.hudi.source.split.HoodieContinuousSplitDiscover;
+import org.apache.hudi.source.split.HoodieSourceSplit;
+import org.apache.hudi.source.split.HoodieSplitProvider;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi 
commits of upstream Hoodie table.
+ */
+public class HoodieContinuousSplitEnumerator extends 
AbstractHoodieSplitEnumerator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class);
+
+  private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext;
+  private final HoodieSplitProvider splitProvider;
+  private final HoodieContinuousSplitDiscover splitDiscover;
+  private final ScanContext scanContext;
+
+  /**
+   * Instant for the last enumerated commit. Next incremental enumeration 
should be based off
+   * this as the starting position.
+   */
+  private final AtomicReference<HoodieEnumeratorPosition> position;
+
+  public HoodieContinuousSplitEnumerator(
+      SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext,
+      HoodieSplitProvider splitProvider,
+      HoodieContinuousSplitDiscover splitDiscover,
+      ScanContext scanContext,
+      @Nullable HoodieSplitEnumeratorState enumState) {
+    super(enumeratorContext, splitProvider);
+    this.enumeratorContext = enumeratorContext;
+    this.splitProvider = splitProvider;
+    this.splitDiscover = splitDiscover;
+    this.scanContext = scanContext;
+    this.position = new AtomicReference<>();
+
+    if (enumState != null) {
+      
this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(),
 enumState.getLastEnumeratedInstantCompletionTime()));
+    } else {
+      
this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), 
scanContext.getStartInstant()));
+    }
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    enumeratorContext.callAsync(
+        this::discoverSplits,
+        this::processDiscoveredSplits,
+        0L,
+        scanContext.getScanInterval().toMillis());
+  }
+
+  @Override
+  boolean shouldWaitForMoreSplits() {
+    return true;
+  }
+
+  @Override
+  public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws 
Exception {
+    return new HoodieSplitEnumeratorState(splitProvider.state(), 
position.get().lastInstant(), position.get().lastInstantCompletionTime());
+  }
+
+  private HoodieContinuousSplitBatch discoverSplits() {
+    int pendingSplitNumber = splitProvider.pendingSplitCount();
+    if (pendingSplitNumber > scanContext.getMaxPendingSplits()) {
+      LOG.info(
+          "Pause split discovery as the assigner already has too many pending 
splits: {}",
+          pendingSplitNumber);
+      return HoodieContinuousSplitBatch.EMPTY;
+    }
+    return 
splitDiscover.discoverSplits(position.get().lastInstantCompletionTime());
+  }
+
+  private void processDiscoveredSplits(HoodieContinuousSplitBatch result, 
Throwable throwable) {
+    if (throwable == null) {

Review Comment:
   ```suggestion
       if (throwable != null) {
   ```
   it's more common to return/end early in a method, then happy path code bring 
outside of code block.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,134 @@
+/*
+ * 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.enumerator;
+
+import org.apache.hudi.source.ScanContext;
+import org.apache.hudi.source.split.HoodieContinuousSplitBatch;
+import org.apache.hudi.source.split.HoodieContinuousSplitDiscover;
+import org.apache.hudi.source.split.HoodieSourceSplit;
+import org.apache.hudi.source.split.HoodieSplitProvider;
+
+import org.apache.flink.api.connector.source.SplitEnumeratorContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import javax.annotation.Nullable;
+
+import java.util.Objects;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Continuous Hoodie enumerator that discovers Hoodie splits from new Hudi 
commits of upstream Hoodie table.
+ */
+public class HoodieContinuousSplitEnumerator extends 
AbstractHoodieSplitEnumerator {
+  private static final Logger LOG = 
LoggerFactory.getLogger(HoodieContinuousSplitEnumerator.class);
+
+  private final SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext;
+  private final HoodieSplitProvider splitProvider;
+  private final HoodieContinuousSplitDiscover splitDiscover;
+  private final ScanContext scanContext;
+
+  /**
+   * Instant for the last enumerated commit. Next incremental enumeration 
should be based off
+   * this as the starting position.
+   */
+  private final AtomicReference<HoodieEnumeratorPosition> position;
+
+  public HoodieContinuousSplitEnumerator(
+      SplitEnumeratorContext<HoodieSourceSplit> enumeratorContext,
+      HoodieSplitProvider splitProvider,
+      HoodieContinuousSplitDiscover splitDiscover,
+      ScanContext scanContext,
+      @Nullable HoodieSplitEnumeratorState enumState) {
+    super(enumeratorContext, splitProvider);
+    this.enumeratorContext = enumeratorContext;
+    this.splitProvider = splitProvider;
+    this.splitDiscover = splitDiscover;
+    this.scanContext = scanContext;
+    this.position = new AtomicReference<>();
+
+    if (enumState != null) {
+      
this.position.set(HoodieEnumeratorPosition.of(enumState.getLastEnumeratedInstant(),
 enumState.getLastEnumeratedInstantCompletionTime()));
+    } else {
+      
this.position.set(HoodieEnumeratorPosition.of(scanContext.getStartInstant(), 
scanContext.getStartInstant()));
+    }
+  }
+
+  @Override
+  public void start() {
+    super.start();
+    enumeratorContext.callAsync(
+        this::discoverSplits,
+        this::processDiscoveredSplits,
+        0L,
+        scanContext.getScanInterval().toMillis());
+  }
+
+  @Override
+  boolean shouldWaitForMoreSplits() {
+    return true;
+  }
+
+  @Override
+  public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws 
Exception {
+    return new HoodieSplitEnumeratorState(splitProvider.state(), 
position.get().lastInstant(), position.get().lastInstantCompletionTime());
+  }
+
+  private HoodieContinuousSplitBatch discoverSplits() {
+    int pendingSplitNumber = splitProvider.pendingSplitCount();
+    if (pendingSplitNumber > scanContext.getMaxPendingSplits()) {
+      LOG.info(
+          "Pause split discovery as the assigner already has too many pending 
splits: {}",
+          pendingSplitNumber);
+      return HoodieContinuousSplitBatch.EMPTY;
+    }
+    return 
splitDiscover.discoverSplits(position.get().lastInstantCompletionTime());
+  }
+
+  private void processDiscoveredSplits(HoodieContinuousSplitBatch result, 
Throwable throwable) {
+    if (throwable == null) {
+      if (!Objects.equals(result.getFromInstant(), 
position.get().lastInstantCompletionTime())) {
+        LOG.info(
+            "Skip {} discovered splits because the scan starting position 
doesn't match "
+                + "the current enumerator position: enumerator position = {}, 
scan starting position = {}",
+            result.getSplits().size(),
+            position.get().lastInstantCompletionTime(),
+            result.getFromInstant());
+      } else {
+        if (!result.getSplits().isEmpty()) {
+          splitProvider.onDiscoveredSplits(result.getSplits());
+          LOG.info(
+              "Added {} splits discovered between ({}, {}] to the assigner",
+              result.getSplits().size(),
+              result.getFromInstant(),
+              result.getToInstant());
+        } else {
+          LOG.info(
+              "No new splits discovered between ({}, {}]",
+              result.getFromInstant(),
+              result.getToInstant());
+        }
+        
position.set(HoodieEnumeratorPosition.of(position.get().lastInstantCompletionTime(),
 result.getToInstant()));

Review Comment:
   can you add a comment to this line and explain this update logic pls?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieSplitEnumeratorState.java:
##########
@@ -20,20 +20,35 @@
 
 import org.apache.hudi.source.split.HoodieSourceSplitState;
 
+import javax.annotation.Nullable;
+
 import java.io.Serializable;
 import java.util.Collection;
 
 /**
  * State of Hoodie split enumerator. Mainly include the states of pending 
splits of split provider.
  */
 public class HoodieSplitEnumeratorState implements Serializable {
+  private final String lastEnumeratedInstant;
+  private final String lastEnumeratedInstantCompletionTime;
   private final Collection<HoodieSourceSplitState> pendingSplitStates;
 
-  public HoodieSplitEnumeratorState(Collection<HoodieSourceSplitState> 
pendingSplitStates) {
+  public HoodieSplitEnumeratorState(
+      Collection<HoodieSourceSplitState> pendingSplitStates, @Nullable String 
lastEnumeratedInstant, @Nullable String lastEnumeratedInstantCompletionTime) {

Review Comment:
   try to stay with hudi's convention of using 
`org.apache.hudi.common.util.Option` over passing nulls, at least for hudi's 
internal models.



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/split/HoodieSourceSplit.java:
##########
@@ -31,8 +31,8 @@
  * Hoodie SourceSplit implementation for source V2.
  */
 public class HoodieSourceSplit implements SourceSplit, Serializable {
+  public static int SPLIT_COUNTER = 0;

Review Comment:
   should prefer `AtomicLong` 



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/IncrementalInputSplits.java:
##########
@@ -303,6 +305,26 @@ public Result inputSplits(
     }
   }
 
+  /**
+   * Returns the incremental Hoodie source split batch.
+   *
+   * @param metaClient    The meta client
+   * @param startInstant  The start Instant of the splits
+   * @param cdcEnabled    Whether cdc is enabled
+   *
+   * @return The list of incremental input splits or empty if there are no new 
instants
+   */
+  public HoodieContinuousSplitBatch inputHoodieSourceSplits(
+      HoodieTableMetaClient metaClient,
+      @Nullable String startInstant,
+      boolean cdcEnabled) {
+    Result result = inputSplits(metaClient, startInstant, cdcEnabled);
+    List<HoodieSourceSplit> splits = result.inputSplits.stream().map(split -> 
new HoodieSourceSplit(

Review Comment:
   pls track any deferred refactoring work in a sub-issue so we won't lose it



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/AbstractHoodieSplitEnumerator.java:
##########
@@ -116,7 +116,7 @@ public void addReader(int subtaskId) {
 
   @Override
   public HoodieSplitEnumeratorState snapshotState(long checkpointId) throws 
Exception {
-    return new HoodieSplitEnumeratorState(splitProvider.state());
+    return new HoodieSplitEnumeratorState(splitProvider.state(), null, null);

Review Comment:
   same here. let's avoid the pattern of passing `null`s



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