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



##########
File path: 
flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/enumerator/ContinuousSplitPlannerImpl.java
##########
@@ -0,0 +1,160 @@
+/*
+ * 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.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.StreamingStartingStrategy;
+import org.apache.iceberg.flink.source.split.IcebergSourceSplit;
+import org.apache.iceberg.util.SnapshotUtil;
+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 ScanContext scanContext;
+  private final ExecutorService workerPool;
+
+  public ContinuousSplitPlannerImpl(Table table, ScanContext scanContext, 
String threadPoolName) {
+    this.table = table;
+    this.scanContext = scanContext;
+    this.workerPool = ThreadPools.newWorkerPool(
+        "iceberg-plan-worker-pool-" + threadPoolName, 
scanContext.planParallelism());
+  }
+
+  @Override
+  public ContinuousEnumerationResult planSplits(IcebergEnumeratorPosition 
lastPosition) {
+    table.refresh();
+    if (lastPosition != null) {
+      return discoverIncrementalSplits(lastPosition);
+    } else {
+      return discoverInitialSplits();
+    }
+  }
+
+  /**
+   * Discover incremental changes between @{code lastPosition} and current 
table snapshot
+   */
+  private ContinuousEnumerationResult 
discoverIncrementalSplits(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 
StreamingStartingStrategy}.
+   *
+   * <li>{@link ContinuousEnumerationResult#splits()} should contain initial 
splits
+   * discovered from table scan for {@link 
StreamingStartingStrategy#TABLE_SCAN_THEN_INCREMENTAL}.
+   * For all other strategies, splits collection should be empty.
+   * <li>{@link ContinuousEnumerationResult#position()} points to the starting 
position
+   * for the next incremental split discovery with exclusive behavior. Meaning 
files committed
+   * by the snapshot from the position in {@code ContinuousEnumerationResult} 
won't be included
+   * in the next incremental scan.
+   */
+  private ContinuousEnumerationResult discoverInitialSplits() {
+    Optional<Snapshot> startSnapshotOptional = getStartSnapshot(table, 
scanContext);
+    if (!startSnapshotOptional.isPresent()) {
+      // set IcebergEnumeratorPosition to null in this case
+      return new ContinuousEnumerationResult(Collections.emptyList(), null);
+    }

Review comment:
       agree it is not just the first snapshot. I was using it just to simplify 
the discussion. it doesn't really matter how many snapshots are committed 
before the first discovery. semantics are the same.
   
   I was saying `earliest` or `latest` snapshot can be interpreted as the 
earliest or latest valid snapshot of the table. Let's say when the Flink job 
started, it is an empty table. initial scan return null. 
   * case 1: there are 3 snapshots (S1, S2, S3) committed. Now Flink source  
ran initial scan again. For earliest strategy, S1 will be used as starting 
snapshot. For latest strategy, S3 will be used as starting snapshot. 
   * case 2: there is 1 snapshot committed. Now both earliest and latest 
strategy will discover S1 as starting snapshot
   
   regarding -1 snapshot id, is it a reserved special id? I wouldn't find the 
usage for snapshot id. Also, `IncrementalDataTableScan` seems to validate the 
start snapshot id. This may not work.
   ```
       Preconditions.checkArgument(
           snapshotIdsRange.contains(newFromSnapshotId),
           "from snapshot id %s not in existing snapshot ids range (%s, %s]",
           newFromSnapshotId, context().fromSnapshotId(), newToSnapshotId);
   ```
   
   @yittg I definitely see your perspective, which is a valid interpretation of 
earliest and latest for empty table. In your interpretation, earliest and 
latest will behave the same (with some imaginary snapshot as starting point). 
all future snapshot should be discovered. To implement this model, we need to 
change `appendsBetween` implementation to support a special starting snapshot 
id, which is invalid and yet indicates a special starting point.
   
   To me, both interpretations are reasonable. I also think for the practical 
purpose, how to interpret earliest and latest strategy for an empty table is 
not that important for streaming job. For most streaming jobs, the bootstrap 
point is not really important, as it is arbitrary anyway.  




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