szehon-ho commented on code in PR #5382:
URL: https://github.com/apache/iceberg/pull/5382#discussion_r936115965


##########
core/src/main/java/org/apache/iceberg/BaseIncrementalChangelogScan.java:
##########
@@ -0,0 +1,188 @@
+/*
+ * 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;
+
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.Map;
+import java.util.Set;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ManifestGroup.CreateTasksFunction;
+import org.apache.iceberg.ManifestGroup.TaskContext;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalChangelogScan
+    extends BaseIncrementalScan<
+        IncrementalChangelogScan, ChangelogScanTask, 
ScanTaskGroup<ChangelogScanTask>>
+    implements IncrementalChangelogScan {
+
+  BaseIncrementalChangelogScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalChangelogScan(
+      TableOperations ops, Table table, Schema schema, TableScanContext 
context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalChangelogScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, 
TableScanContext newContext) {
+    return new BaseIncrementalChangelogScan(newOps, newTable, newSchema, 
newContext);
+  }
+
+  @Override
+  protected CloseableIterable<ChangelogScanTask> doPlanFiles(
+      Long fromSnapshotIdExclusive, long toSnapshotIdInclusive) {
+
+    Deque<Snapshot> changelogSnapshots =

Review Comment:
   Opt/Nit:  It seems a bit strange to have a deque so early, when I see its 
really needed inside CreateDataFileChangeTasks.  You have to kind of track the 
code in two places to see how it works.
   
   What do you think about, just using Iterable<Snapshot>  here and then only 
doing the Deque / computeSnapshotOrdinals logic inside the  
CreateDataFileChangeTasks constructor?
   
   The slight negative might be , having to transform Iterable<Snapshot> to 
stream for some of these methods.



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalScan.java:
##########
@@ -0,0 +1,140 @@
+/*
+ * 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;
+
+import org.apache.iceberg.events.IncrementalScanEvent;
+import org.apache.iceberg.events.Listeners;
+import org.apache.iceberg.io.CloseableIterable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.util.SnapshotUtil;
+
+abstract class BaseIncrementalScan<ThisT, T extends ScanTask, G extends 
ScanTaskGroup<T>>
+    extends BaseScan<ThisT, T, G> implements IncrementalScan<ThisT, T, G> {
+
+  protected BaseIncrementalScan(
+      TableOperations ops, Table table, Schema schema, TableScanContext 
context) {
+    super(ops, table, schema, context);
+  }
+
+  protected abstract CloseableIterable<T> doPlanFiles(
+      Long fromSnapshotIdExclusive, long toSnapshotIdInclusive);
+
+  @Override
+  public ThisT fromSnapshotInclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(
+        table().snapshot(fromSnapshotId) != null,
+        "Cannot find the starting snapshot: %s",
+        fromSnapshotId);
+    TableScanContext newContext = 
context().fromSnapshotIdInclusive(fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), newContext);
+  }
+
+  @Override
+  public ThisT fromSnapshotExclusive(long fromSnapshotId) {
+    // for exclusive behavior, table().snapshot(fromSnapshotId) check can't be 
applied
+    // as fromSnapshotId could be matched to a parent snapshot that is already 
expired
+    TableScanContext newContext = 
context().fromSnapshotIdExclusive(fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), newContext);
+  }
+
+  @Override
+  public ThisT toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(
+        table().snapshot(toSnapshotId) != null, "Cannot find the end snapshot: 
%s", toSnapshotId);
+    TableScanContext newContext = context().toSnapshotId(toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), newContext);
+  }
+
+  @Override
+  public CloseableIterable<T> planFiles() {
+    if (scanCurrentLineage() && table().currentSnapshot() == null) {
+      // If the table is empty (no current snapshot) and both from and to 
snapshots aren't set,
+      // simply return an empty iterable. In this case, the listener 
notification is also skipped.
+      return CloseableIterable.empty();
+    }
+
+    long toSnapshotIdInclusive = toSnapshotIdInclusive();
+    Long fromSnapshotIdExclusive = 
fromSnapshotIdExclusive(toSnapshotIdInclusive);
+
+    if (fromSnapshotIdExclusive != null) {
+      Listeners.notifyAll(
+          new IncrementalScanEvent(
+              table().name(),
+              fromSnapshotIdExclusive,
+              toSnapshotIdInclusive,
+              filter(),
+              schema(),
+              false));
+    } else {
+      Listeners.notifyAll(
+          new IncrementalScanEvent(
+              table().name(),
+              SnapshotUtil.oldestAncestorOf(table(), 
toSnapshotIdInclusive).snapshotId(),
+              toSnapshotIdInclusive,
+              filter(),
+              schema(),
+              true));
+    }
+
+    return doPlanFiles(fromSnapshotIdExclusive, toSnapshotIdInclusive);
+  }
+
+  private boolean scanCurrentLineage() {
+    return context().fromSnapshotId() == null && context().toSnapshotId() == 
null;
+  }
+
+  private long toSnapshotIdInclusive() {
+    if (context().toSnapshotId() != null) {
+      return context().toSnapshotId();
+    } else {
+      Snapshot currentSnapshot = table().currentSnapshot();
+      Preconditions.checkArgument(
+          currentSnapshot != null, "End snapshot is not set and table has no 
current snapshot");
+      return currentSnapshot.snapshotId();
+    }
+  }
+
+  private Long fromSnapshotIdExclusive(long toSnapshotIdInclusive) {
+    Long fromSnapshotId = context().fromSnapshotId();
+
+    if (fromSnapshotId == null) {
+      return null;
+
+    } else if (context().fromSnapshotInclusive()) {

Review Comment:
   Nit: java style question (as you have probably read more about it than me), 
what do you think , is it better to just start another if block instead of 
doing 'else-if' in this case, because the above block is a return?  I 
personally have a hard time understanding multi if-else-else cases where the 
conditions are on different variables. 



##########
api/src/main/java/org/apache/iceberg/Scan.java:
##########
@@ -137,6 +144,13 @@
    */
   CloseableIterable<G> planTasks();
 
+  /**
+   * Returns whether this scan is case-sensitive with respect to column names.
+   *
+   * @return true if case-sensitive, false otherwise.
+   */
+  boolean isCaseSensitive();

Review Comment:
   Nit: Realize its a move, but maybe we could move the caseSensitive() getter 
up to the setter, or just put the other getter (filter) to the bottom of the 
file?



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