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


##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/ScanContext.java:
##########
@@ -0,0 +1,197 @@
+/*
+ * 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;
+
+import org.apache.hudi.configuration.FlinkOptions;
+
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.configuration.Configuration;
+import org.apache.flink.core.fs.Path;
+import org.apache.flink.table.types.logical.RowType;
+
+import java.io.Serializable;
+import java.time.Duration;
+
+/**
+ * Hudi source scan context.
+ */
+@Internal
+public class ScanContext implements Serializable {
+  private final Configuration conf;
+  private final Path path;
+  private final RowType rowType;
+  private final String startInstant;
+  private final long maxCompactionMemoryInBytes;
+  private final long maxPendingSplits;
+  // skip compaction
+  private final boolean skipCompaction;
+  // skip clustering
+  private final boolean skipClustering;
+  // skip insert overwrite
+  private final boolean skipInsertOverwrite;
+  // cdc enabled
+  private final boolean cdcEnabled;

Review Comment:
   @HuangZhenQiu let's pls note down follow-up refactoring items in a subissue 
under https://github.com/apache/hudi/issues/17038 ?



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/split/HoodieContinuousSplitBatch.java:
##########
@@ -0,0 +1,63 @@
+/*
+ * 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.split;
+
+import org.apache.flink.util.Preconditions;
+
+import java.util.Collection;
+import java.util.Collections;
+
+/**
+ * Result from continuous enumerator.
+ */
+public class HoodieContinuousSplitBatch {
+  public static final HoodieContinuousSplitBatch EMPTY = new 
HoodieContinuousSplitBatch(Collections.emptyList(), "", "");
+  private final Collection<HoodieSourceSplit> splits;
+  private final String fromInstant;
+  private final String toInstant;
+
+  /**
+   * @param splits should never be null. But it can be an empty collection
+   * @param fromInstant could be null
+   * @param toInstant should never be null. But it can be empty string
+   */
+  public HoodieContinuousSplitBatch(
+      Collection<HoodieSourceSplit> splits,
+      String fromInstant,
+      String toInstant) {
+
+    Preconditions.checkArgument(splits != null, "Invalid to splits collection: 
null");
+    Preconditions.checkArgument(toInstant != null, "Invalid end instant: 
null");

Review Comment:
   use `org.apache.hudi.common.util.ValidationUtils` for similar purpose



##########
hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/source/enumerator/HoodieContinuousSplitEnumerator.java:
##########
@@ -0,0 +1,133 @@
+/*
+ * 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.common.util.Option;
+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 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,
+      Option<HoodieSplitEnumeratorState> enumStateOpt) {
+    super(enumeratorContext, splitProvider);
+    this.enumeratorContext = enumeratorContext;
+    this.splitProvider = splitProvider;
+    this.splitDiscover = splitDiscover;
+    this.scanContext = scanContext;
+    this.position = new AtomicReference<>();
+
+    if (enumStateOpt.isPresent()) {
+      
this.position.set(HoodieEnumeratorPosition.of(enumStateOpt.get().getLastEnumeratedInstant(),
 enumStateOpt.get().getLastEnumeratedInstantOffset()));
+    } 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().issuedInstant(), position.get().issuedOffset());
+  }
+
+  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().issuedOffset().isPresent() ? 
position.get().issuedOffset().get() : null);
+  }
+
+  private void processDiscoveredSplits(HoodieContinuousSplitBatch result, 
Throwable throwable) {
+    if (throwable != null) {
+      throw new RuntimeException("Failed to discover new splits", throwable);
+    } else {

Review Comment:
   for simpler reading, this else block is not needed



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