stevenzwu commented on a change in pull request #4329:
URL: https://github.com/apache/iceberg/pull/4329#discussion_r828133443



##########
File path: 
flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
##########
@@ -0,0 +1,184 @@
+/*
+ * 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.iceberg.flink.source.enumerator;
+
+import java.util.Collections;
+import java.util.List;
+import java.util.Optional;
+import java.util.concurrent.ExecutorService;
+import org.apache.flink.annotation.Internal;
+import org.apache.flink.annotation.VisibleForTesting;
+import org.apache.iceberg.HistoryEntry;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.flink.source.FlinkSplitPlanner;
+import org.apache.iceberg.flink.source.ScanContext;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.ThreadPools;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Internal
+public class ContinuousSplitPlannerImpl implements ContinuousSplitPlanner {
+  private static final Logger LOG = 
LoggerFactory.getLogger(ContinuousSplitPlannerImpl.class);
+
+  private final Table table;
+  private final IcebergEnumeratorConfig config;
+  private final ScanContext scanContext;
+  private final ExecutorService workerPool;
+
+  public ContinuousSplitPlannerImpl(Table table, IcebergEnumeratorConfig 
config, ScanContext scanContext) {
+    this.table = table;
+    this.config = config;
+    this.scanContext = scanContext;
+    // Within a JVM, table name should be unique across sources.
+    // Hence it is used as worker pool thread name prefix.
+    this.workerPool = ThreadPools.newWorkerPool("iceberg-worker-pool-" + 
table.name(), scanContext.planParallelism());
+    validate();
+  }
+
+  private void validate() {
+    Preconditions.checkArgument(scanContext.snapshotId() == null,
+        "Can't set snapshotId in ScanContext for continuous enumerator");
+    Preconditions.checkArgument(scanContext.asOfTimestamp() == null,
+        "Can't set asOfTimestamp in ScanContext for continuous enumerator");
+    Preconditions.checkArgument(scanContext.startSnapshotId() == null,
+        "Can't set startSnapshotId in ScanContext for continuous enumerator");
+    Preconditions.checkArgument(scanContext.endSnapshotId() == null,
+        "Can't set endSnapshotId in ScanContext for continuous enumerator");
+  }
+
+  @Override
+  public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition 
lastPosition) {
+    table.refresh();
+    if (lastPosition != null) {
+      return discoverDeltaSplits(lastPosition);
+    } else {
+      return discoverInitialSplits();
+    }
+  }
+
+  /**
+   * Discover delta changes between @{code lastPosition} and current table 
snapshot
+   */
+  private ContinuousEnumerationResult 
discoverDeltaSplits(IcebergEnumeratorPosition lastPosition) {
+    // incremental discovery mode
+    Snapshot currentSnapshot = table.currentSnapshot();
+    if (currentSnapshot.snapshotId() == lastPosition.endSnapshotId()) {
+      LOG.info("Current table snapshot is already enumerated: {}", 
currentSnapshot.snapshotId());
+      return new ContinuousEnumerationResult(Collections.emptyList(), 
lastPosition);
+    } else {
+      ScanContext incrementalScan = scanContext
+          .copyWithAppendsBetween(lastPosition.endSnapshotId(), 
currentSnapshot.snapshotId());
+      LOG.info("Incremental scan: startSnapshotId = {}, endSnapshotId = {}",
+          incrementalScan.startSnapshotId(), incrementalScan.endSnapshotId());
+      List<IcebergSourceSplit> splits = 
FlinkSplitPlanner.planIcebergSourceSplits(table, incrementalScan, workerPool);
+      IcebergEnumeratorPosition position = IcebergEnumeratorPosition.builder()
+          .startSnapshotId(lastPosition.endSnapshotId())
+          .startSnapshotTimestampMs(lastPosition.endSnapshotTimestampMs())
+          .endSnapshotId(currentSnapshot.snapshotId())
+          .endSnapshotTimestampMs(currentSnapshot.timestampMillis())
+          .build();
+      LOG.info("Discovered {} splits from incremental scan: {}", 
splits.size(), position);
+      return new ContinuousEnumerationResult(splits, position);
+    }
+  }
+
+  /**
+   * Discovery initial set of splits based on {@link 
IcebergEnumeratorConfig.StartingStrategy}
+   */
+  private ContinuousEnumerationResult discoverInitialSplits() {
+    HistoryEntry startSnapshotEntry = getStartSnapshot(table, config);
+    LOG.info("get startSnapshotId {} based on starting strategy {}",
+        startSnapshotEntry.snapshotId(), config.startingStrategy());
+    List<IcebergSourceSplit> splits;
+    if (config.startingStrategy() ==
+        IcebergEnumeratorConfig.StartingStrategy.TABLE_SCAN_THEN_INCREMENTAL) {
+      // do a full table scan first
+      splits = FlinkSplitPlanner.planIcebergSourceSplits(table, scanContext, 
workerPool);
+      LOG.info("Discovered {} splits from initial full table scan with 
snapshotId {}",
+          splits.size(), startSnapshotEntry);
+    } else {
+      splits = Collections.emptyList();
+    }
+
+    IcebergEnumeratorPosition position = IcebergEnumeratorPosition.builder()
+        .endSnapshotId(startSnapshotEntry.snapshotId())
+        .endSnapshotTimestampMs(startSnapshotEntry.timestampMillis())
+        .build();
+    return new ContinuousEnumerationResult(splits, position);
+  }
+
+  @VisibleForTesting
+  static HistoryEntry getStartSnapshot(Table table, IcebergEnumeratorConfig 
enumeratorConfig) {

Review comment:
       start snapshot is exclusive. I am thinking that we can document the 
behavior better for `StartingStrategy` enum class.




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



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to