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



##########
File path: 
flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -114,77 +128,104 @@ private ScanContext(boolean caseSensitive, Long 
snapshotId, Long startSnapshotId
     this.includeColumnStats = includeColumnStats;
     this.exposeLocality = exposeLocality;
     this.planParallelism = planParallelism;
+
+    validate();
+  }
+
+  private void validate() {
+    if (isStreaming) {

Review comment:
       Will it be allowed to set any `startingStrategy` value if it's in batch 
mode ?

##########
File path: 
flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/ScanContext.java
##########
@@ -114,77 +128,104 @@ private ScanContext(boolean caseSensitive, Long 
snapshotId, Long startSnapshotId
     this.includeColumnStats = includeColumnStats;
     this.exposeLocality = exposeLocality;
     this.planParallelism = planParallelism;
+
+    validate();
+  }
+
+  private void validate() {
+    if (isStreaming) {
+      if (startingStrategy == 
StreamingStartingStrategy.SPECIFIC_START_SNAPSHOT_ID) {

Review comment:
       Should we also consider the validation for other `strategy`s ? 
   
   * `TABLE_SCAN_THEN_INCREMENTAL`
   
   Will the `startSnapshotId`  or `startSnapshotTimestamp` specify the first 
snapshot to do the regular table scan ?  Is there any meaning to set the 
`startSnapshotTimestamp` ? 
   
   * LATEST_SNAPSHOT
   
   As we always read data from the latest available snapshot, so is there any 
meaning to set the `startSnapshotId`,  `startSnapshotTimestamp`, 
`endSnapshotId`, `asOfTimestamp` ? 
   
   * EARLIEST_SNAPSHOT
   
   Ditto
   
   
   Besides,  do we expect the streaming strategy to be a bounded stream in 
which case we may need to set the `endSnapshotId`, or a unbounded stream ? 

##########
File path: 
flink/v1.14/flink/src/main/java/org/apache/iceberg/flink/source/StreamingStartingStrategy.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+/**
+ * Starting strategy for streaming execution. Note that starting snapshot is 
exclusive in the incremental
+ * split discovery mode. Files appended by the starting snapshot aren't 
included in split discovery.
+ */
+public enum StreamingStartingStrategy {
+  /**
+   * First do a regular table scan. then switch to incremental mode.
+   */
+  TABLE_SCAN_THEN_INCREMENTAL,
+
+  /**
+   * Start incremental mode from the latest snapshot
+   */
+  LATEST_SNAPSHOT,

Review comment:
       Agreed.   It's more clear to add the `INCREMENTAL` prefix to align with 
the `TABLE_SCAN_THEN_INCREMENTAL`.




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