You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/05/27 19:56:37 UTC

[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4744: Core: Incremental append scan impl

aokolnychyi commented on code in PR #4744:
URL: https://github.com/apache/iceberg/pull/4744#discussion_r883896076


##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.expressions.Binder;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+abstract class BaseScan<T extends Scan<T>> implements Scan<T> {
+  private final TableOperations ops;
+  private final Table table;
+  private final Schema schema;
+  private final TableScanContext context;
+
+  protected BaseScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    this.ops = ops;
+    this.table = table;
+    this.schema = schema;
+    this.context = context;
+  }
+
+  protected TableOperations tableOps() {
+    return ops;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  protected Schema tableSchema() {
+    return schema;
+  }
+
+  protected TableScanContext context() {
+    return context;
+  }
+
+  protected abstract T newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext);
+
+  @Override
+  public T option(String property, String value) {
+    return newRefinedScan(ops, table, schema, context.withOption(property, value));
+  }
+
+  @Override
+  public T project(Schema projectedSchema) {
+    return newRefinedScan(ops, table, schema, context.project(projectedSchema));
+  }
+
+  @Override
+  public T caseSensitive(boolean caseSensitive) {
+    return newRefinedScan(ops, table, schema, context.setCaseSensitive(caseSensitive));
+  }
+
+  @Override
+  public T includeColumnStats() {
+    return newRefinedScan(ops, table, schema, context.shouldReturnColumnStats(true));
+  }
+
+  @Override
+  public T select(Collection<String> columns) {
+    return newRefinedScan(ops, table, schema, context.selectColumns(columns));
+  }
+
+  @Override
+  public T filter(Expression expr) {
+    return newRefinedScan(ops, table, schema, context.filterRows(Expressions.and(context.rowFilter(), expr)));
+  }
+
+  @Override
+  public T ignoreResiduals() {
+    return newRefinedScan(ops, table, schema, context.ignoreResiduals(true));
+  }
+
+  @Override
+  public T planWith(ExecutorService executorService) {
+    return newRefinedScan(ops, table, schema, context.planWith(executorService));
+  }
+
+  @Override
+  public Schema schema() {
+    return lazyColumnProjection(context, schema);
+  }
+
+  @Override
+  public long targetSplitSize() {

Review Comment:
   Not directly related to this PR but can we remove the override of this method in `DataTableScan`?



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {

Review Comment:
   How do we handle schema evolution in incremental scans? Do we always pick the current schema? Do we test it anywhere?



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {

Review Comment:
   nit: arg formatting



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(fromSnapshotId) != null,
+        "Cannot find the starting snapshot: %d", fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotIdInclusive(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan 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
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(toSnapshotId) != null,
+        "Cannot find end snapshot: %d", toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(),  context().toSnapshotId(toSnapshotId));
+  }
+
+  @Override
+  public CloseableIterable<FileScanTask> planFiles() {
+    long toSnapshotIdInclusive;
+    if (context().toSnapshotId() != null) {
+      toSnapshotIdInclusive = context().toSnapshotId();
+    } else if (table().currentSnapshot() != null) {
+      toSnapshotIdInclusive = table().currentSnapshot().snapshotId();
+    } else if (context().fromSnapshotId() != null) {
+      throw new IllegalArgumentException(
+          "Invalid config: end snapshot is not set, start snapshot is set, and table has no current snapshot");
+    } else {
+      // It is an empty table (no current snapshot). Both from and to snapshots aren't set either.
+      // In this case, listener notification is also skipped
+      return CloseableIterable.empty();
+    }
+
+    // fromSnapshotIdExclusive can be null. appendsBetween handles null fromSnapshotIdExclusive properly
+    // by finding the oldest ancestor of end snapshot.
+    Long fromSnapshotIdExclusive = context().fromSnapshotId();
+    if (context().fromSnapshotId() != null) {

Review Comment:
   Looks like this block accesses the same variables from `context` quite a few times. Would it make sense to define temp vars? It may make it a little bit easier to read as there will be fewer method calls.



##########
core/src/main/java/org/apache/iceberg/util/SnapshotUtil.java:
##########
@@ -107,6 +120,26 @@ public static Snapshot oldestAncestor(Table table) {
     return lastSnapshot;
   }
 
+  /**
+   * Traverses the history and finds the oldest ancestor of the specified snapshot.
+   * <p>
+   * Oldest ancestor is defined as the ancestor snapshot whose parent is null or has been expired.
+   * If the specified snapshot has no parent or parent has been expired,
+   * the specified snapshot itself is returned.
+   *
+   * @param snapshotId the ID of the snapshot to find the oldest ancestor
+   * @param lookup lookup function from snapshot ID to snapshot
+   * @return null if there is no current snapshot in the table, else the oldest Snapshot.
+   */
+  public static Snapshot oldestAncestor(long snapshotId, Function<Long, Snapshot> lookup) {

Review Comment:
   Question: shall we call it `oldestAncestorOf`? I guess other methods in this class use this pattern when a snapshot ID is provided explicitly.



##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.expressions.Binder;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+abstract class BaseScan<T extends Scan<T>> implements Scan<T> {
+  private final TableOperations ops;
+  private final Table table;
+  private final Schema schema;
+  private final TableScanContext context;
+
+  protected BaseScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    this.ops = ops;
+    this.table = table;
+    this.schema = schema;
+    this.context = context;
+  }
+
+  protected TableOperations tableOps() {
+    return ops;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  protected Schema tableSchema() {
+    return schema;
+  }
+
+  protected TableScanContext context() {
+    return context;
+  }
+
+  protected abstract T newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext);
+
+  @Override
+  public T option(String property, String value) {
+    return newRefinedScan(ops, table, schema, context.withOption(property, value));
+  }
+
+  @Override
+  public T project(Schema projectedSchema) {
+    return newRefinedScan(ops, table, schema, context.project(projectedSchema));
+  }
+
+  @Override
+  public T caseSensitive(boolean caseSensitive) {
+    return newRefinedScan(ops, table, schema, context.setCaseSensitive(caseSensitive));
+  }
+
+  @Override
+  public T includeColumnStats() {
+    return newRefinedScan(ops, table, schema, context.shouldReturnColumnStats(true));
+  }
+
+  @Override
+  public T select(Collection<String> columns) {
+    return newRefinedScan(ops, table, schema, context.selectColumns(columns));
+  }
+
+  @Override
+  public T filter(Expression expr) {
+    return newRefinedScan(ops, table, schema, context.filterRows(Expressions.and(context.rowFilter(), expr)));
+  }
+
+  @Override
+  public T ignoreResiduals() {
+    return newRefinedScan(ops, table, schema, context.ignoreResiduals(true));
+  }
+
+  @Override
+  public T planWith(ExecutorService executorService) {
+    return newRefinedScan(ops, table, schema, context.planWith(executorService));
+  }
+
+  @Override
+  public Schema schema() {
+    return lazyColumnProjection(context, schema);
+  }
+
+  @Override
+  public long targetSplitSize() {
+    long tableValue = ops.current().propertyAsLong(
+        TableProperties.SPLIT_SIZE,
+        TableProperties.SPLIT_SIZE_DEFAULT);
+    return PropertyUtil.propertyAsLong(context.options(), TableProperties.SPLIT_SIZE, tableValue);
+  }
+
+  @Override
+  public int splitLookback() {
+    int tableValue = ops.current().propertyAsInt(
+        TableProperties.SPLIT_LOOKBACK,
+        TableProperties.SPLIT_LOOKBACK_DEFAULT);
+    return PropertyUtil.propertyAsInt(context.options(), TableProperties.SPLIT_LOOKBACK, tableValue);
+  }
+
+  @Override
+  public long splitOpenFileCost() {
+    long tableValue = ops.current().propertyAsLong(
+        TableProperties.SPLIT_OPEN_FILE_COST,
+        TableProperties.SPLIT_OPEN_FILE_COST_DEFAULT);
+    return PropertyUtil.propertyAsLong(context.options(), TableProperties.SPLIT_OPEN_FILE_COST, tableValue);
+  }
+
+  /**
+   * Resolve the schema to be projected lazily.
+   *
+   * if there are selected columns from scan context, selected columns are projected to the table schema.

Review Comment:
   nit: `if` -> `If`



##########
core/src/main/java/org/apache/iceberg/TableScanContext.java:
##########
@@ -168,7 +172,17 @@ Long fromSnapshotId() {
   TableScanContext fromSnapshotId(long id) {
     return new TableScanContext(snapshotId, rowFilter, ignoreResiduals,
         caseSensitive, colStats, projectedSchema, selectedColumns, options, id, toSnapshotId,
-        planExecutor);
+        planExecutor, fromSnapshotInclusive);
+  }
+
+  TableScanContext fromSnapshotIdInclusive(long id) {

Review Comment:
   Shall we rename the existing methods (`fromSnapshotId`) to indicate they configure the start snapshot exclusive?
   We may also set `fromSnapshotInclusive` explicitly in existing methods.



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(fromSnapshotId) != null,
+        "Cannot find the starting snapshot: %d", fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotIdInclusive(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan 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
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(toSnapshotId) != null,
+        "Cannot find end snapshot: %d", toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(),  context().toSnapshotId(toSnapshotId));
+  }
+
+  @Override
+  public CloseableIterable<FileScanTask> planFiles() {
+    long toSnapshotIdInclusive;
+    if (context().toSnapshotId() != null) {
+      toSnapshotIdInclusive = context().toSnapshotId();
+    } else if (table().currentSnapshot() != null) {
+      toSnapshotIdInclusive = table().currentSnapshot().snapshotId();
+    } else if (context().fromSnapshotId() != null) {
+      throw new IllegalArgumentException(
+          "Invalid config: end snapshot is not set, start snapshot is set, and table has no current snapshot");
+    } else {
+      // It is an empty table (no current snapshot). Both from and to snapshots aren't set either.
+      // In this case, listener notification is also skipped
+      return CloseableIterable.empty();

Review Comment:
   It is not very obvious to have a return statement for the whole method nested in a block that initializes the end snapshot.
   
   Have you considered a separate method for computing the end snapshot, a call to `Preconditions` and a separate if statement for an early return?



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(fromSnapshotId) != null,
+        "Cannot find the starting snapshot: %d", fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotIdInclusive(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan 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
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(toSnapshotId) != null,
+        "Cannot find end snapshot: %d", toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(),  context().toSnapshotId(toSnapshotId));
+  }
+
+  @Override
+  public CloseableIterable<FileScanTask> planFiles() {
+    long toSnapshotIdInclusive;
+    if (context().toSnapshotId() != null) {
+      toSnapshotIdInclusive = context().toSnapshotId();
+    } else if (table().currentSnapshot() != null) {
+      toSnapshotIdInclusive = table().currentSnapshot().snapshotId();
+    } else if (context().fromSnapshotId() != null) {
+      throw new IllegalArgumentException(
+          "Invalid config: end snapshot is not set, start snapshot is set, and table has no current snapshot");
+    } else {
+      // It is an empty table (no current snapshot). Both from and to snapshots aren't set either.
+      // In this case, listener notification is also skipped
+      return CloseableIterable.empty();
+    }
+
+    // fromSnapshotIdExclusive can be null. appendsBetween handles null fromSnapshotIdExclusive properly
+    // by finding the oldest ancestor of end snapshot.
+    Long fromSnapshotIdExclusive = context().fromSnapshotId();
+    if (context().fromSnapshotId() != null) {
+      if (context().fromSnapshotInclusive()) {
+        // validate the fromSnapshotId is an ancestor of toSnapshotId
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
+            "Starting snapshot (inclusive) %s is not an ancestor of end snapshot %s",
+            context().fromSnapshotId(), toSnapshotIdInclusive);
+        // for inclusive behavior fromSnapshotIdExclusive is set to the parent snapshot id, which can be null.
+        fromSnapshotIdExclusive = table().snapshot(context().fromSnapshotId()).parentId();
+      } else {
+        // validate the parent snapshot id an ancestor of toSnapshotId
+        Preconditions.checkArgument(
+            SnapshotUtil.isParentAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
+            "Starting snapshot (exclusive) %s is not a parent ancestor of end snapshot %s",
+            context().fromSnapshotId(), toSnapshotIdInclusive);
+      }
+    }
+
+    if (fromSnapshotIdExclusive != null) {
+      Listeners.notifyAll(new IncrementalScanEvent(table().name(), fromSnapshotIdExclusive,

Review Comment:
   nit: The purpose of the true/false flags is not clear without looking into the code. Would it make sense to offer static methods like `fromSnapshotInclusive` and `fromSnapshotExclusive` in `IncrementalScanEvent`?



##########
core/src/main/java/org/apache/iceberg/BaseScan.java:
##########
@@ -0,0 +1,171 @@
+/*
+ * 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.Collection;
+import java.util.Collections;
+import java.util.Set;
+import java.util.concurrent.ExecutorService;
+import org.apache.iceberg.expressions.Binder;
+import org.apache.iceberg.expressions.Expression;
+import org.apache.iceberg.expressions.Expressions;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.types.TypeUtil;
+import org.apache.iceberg.util.PropertyUtil;
+
+abstract class BaseScan<T extends Scan<T>> implements Scan<T> {
+  private final TableOperations ops;
+  private final Table table;
+  private final Schema schema;
+  private final TableScanContext context;
+
+  protected BaseScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    this.ops = ops;
+    this.table = table;
+    this.schema = schema;
+    this.context = context;
+  }
+
+  protected TableOperations tableOps() {
+    return ops;
+  }
+
+  protected Table table() {
+    return table;
+  }
+
+  protected Schema tableSchema() {
+    return schema;
+  }
+
+  protected TableScanContext context() {
+    return context;
+  }
+
+  protected abstract T newRefinedScan(

Review Comment:
   nit: arg formatting
   
   It is not super consistent in the code but most places would format args like this:
   
   <img width="737" alt="image" src="https://user-images.githubusercontent.com/6235869/170771523-96c9060d-19e5-4927-be56-f505c291de32.png">



##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -45,70 +38,46 @@
 /**
  * Base class for {@link TableScan} implementations.
  */
-abstract class BaseTableScan implements TableScan {
+abstract class BaseTableScan extends BaseScan<TableScan> implements TableScan {
   private static final Logger LOG = LoggerFactory.getLogger(BaseTableScan.class);
 
-  private final TableOperations ops;
-  private final Table table;
-  private final Schema schema;
-  private final TableScanContext context;
-
   protected BaseTableScan(TableOperations ops, Table table, Schema schema) {
     this(ops, table, schema, new TableScanContext());
   }
 
   protected BaseTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
-    this.ops = ops;
-    this.table = table;
-    this.schema = schema;
-    this.context = context;
-  }
-
-  protected TableOperations tableOps() {
-    return ops;
-  }
-
-  protected Schema tableSchema() {
-    return schema;
+    super(ops, table, schema, context);
   }
 
   protected Long snapshotId() {
-    return context.snapshotId();
+    return context().snapshotId();
   }
 
   protected boolean colStats() {
-    return context.returnColumnStats();
+    return context().returnColumnStats();
   }
 
   protected boolean shouldIgnoreResiduals() {
-    return context.ignoreResiduals();
+    return context().ignoreResiduals();
   }
 
   protected Collection<String> selectedColumns() {
-    return context.selectedColumns();
+    return context().selectedColumns();
   }
 
   protected ExecutorService planExecutor() {
-    return context.planExecutor();
+    return context().planExecutor();
   }
 
   protected Map<String, String> options() {
-    return context.options();
-  }
-
-  protected TableScanContext context() {
-    return context;
+    return context().options();
   }
 
-  @SuppressWarnings("checkstyle:HiddenField")
-  protected abstract TableScan newRefinedScan(TableOperations ops, Table table, Schema schema,
-                                              TableScanContext context);
-
   protected abstract CloseableIterable<FileScanTask> doPlanFiles();
 
   @Override
   public Table table() {

Review Comment:
   Do we still need this implementation here? Can't we consume from the parent directly?



##########
core/src/main/java/org/apache/iceberg/BaseTableScan.java:
##########
@@ -45,70 +38,46 @@
 /**
  * Base class for {@link TableScan} implementations.
  */
-abstract class BaseTableScan implements TableScan {
+abstract class BaseTableScan extends BaseScan<TableScan> implements TableScan {
   private static final Logger LOG = LoggerFactory.getLogger(BaseTableScan.class);
 
-  private final TableOperations ops;
-  private final Table table;
-  private final Schema schema;
-  private final TableScanContext context;
-
   protected BaseTableScan(TableOperations ops, Table table, Schema schema) {
     this(ops, table, schema, new TableScanContext());
   }
 
   protected BaseTableScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
-    this.ops = ops;
-    this.table = table;
-    this.schema = schema;
-    this.context = context;
-  }
-
-  protected TableOperations tableOps() {
-    return ops;
-  }
-
-  protected Schema tableSchema() {
-    return schema;
+    super(ops, table, schema, context);
   }
 
   protected Long snapshotId() {
-    return context.snapshotId();
+    return context().snapshotId();
   }
 
   protected boolean colStats() {
-    return context.returnColumnStats();
+    return context().returnColumnStats();
   }
 
   protected boolean shouldIgnoreResiduals() {
-    return context.ignoreResiduals();
+    return context().ignoreResiduals();
   }
 
   protected Collection<String> selectedColumns() {

Review Comment:
   Is this still being used?



##########
core/src/main/java/org/apache/iceberg/BaseIncrementalAppendScan.java:
##########
@@ -0,0 +1,179 @@
+/*
+ * 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.List;
+import java.util.Set;
+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.relocated.com.google.common.collect.FluentIterable;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterables;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Sets;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.iceberg.util.TableScanUtil;
+
+class BaseIncrementalAppendScan extends BaseScan<IncrementalAppendScan> implements IncrementalAppendScan {
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table) {
+    this(ops, table, table.schema(), new TableScanContext());
+  }
+
+  BaseIncrementalAppendScan(TableOperations ops, Table table, Schema schema, TableScanContext context) {
+    super(ops, table, schema, context);
+  }
+
+  @Override
+  protected IncrementalAppendScan newRefinedScan(
+      TableOperations newOps, Table newTable, Schema newSchema, TableScanContext newContext) {
+    return new BaseIncrementalAppendScan(newOps, newTable, newSchema, newContext);
+  }
+
+  @Override
+  public IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(fromSnapshotId) != null,
+        "Cannot find the starting snapshot: %d", fromSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotIdInclusive(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan 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
+    return newRefinedScan(tableOps(), table(), schema(), context().fromSnapshotId(fromSnapshotId));
+  }
+
+  @Override
+  public IncrementalAppendScan toSnapshot(long toSnapshotId) {
+    Preconditions.checkArgument(table().snapshot(toSnapshotId) != null,
+        "Cannot find end snapshot: %d", toSnapshotId);
+    return newRefinedScan(tableOps(), table(), schema(),  context().toSnapshotId(toSnapshotId));
+  }
+
+  @Override
+  public CloseableIterable<FileScanTask> planFiles() {
+    long toSnapshotIdInclusive;
+    if (context().toSnapshotId() != null) {
+      toSnapshotIdInclusive = context().toSnapshotId();
+    } else if (table().currentSnapshot() != null) {
+      toSnapshotIdInclusive = table().currentSnapshot().snapshotId();
+    } else if (context().fromSnapshotId() != null) {
+      throw new IllegalArgumentException(
+          "Invalid config: end snapshot is not set, start snapshot is set, and table has no current snapshot");
+    } else {
+      // It is an empty table (no current snapshot). Both from and to snapshots aren't set either.
+      // In this case, listener notification is also skipped
+      return CloseableIterable.empty();
+    }
+
+    // fromSnapshotIdExclusive can be null. appendsBetween handles null fromSnapshotIdExclusive properly
+    // by finding the oldest ancestor of end snapshot.
+    Long fromSnapshotIdExclusive = context().fromSnapshotId();
+    if (context().fromSnapshotId() != null) {
+      if (context().fromSnapshotInclusive()) {
+        // validate the fromSnapshotId is an ancestor of toSnapshotId
+        Preconditions.checkArgument(
+            SnapshotUtil.isAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
+            "Starting snapshot (inclusive) %s is not an ancestor of end snapshot %s",
+            context().fromSnapshotId(), toSnapshotIdInclusive);
+        // for inclusive behavior fromSnapshotIdExclusive is set to the parent snapshot id, which can be null.
+        fromSnapshotIdExclusive = table().snapshot(context().fromSnapshotId()).parentId();
+      } else {
+        // validate the parent snapshot id an ancestor of toSnapshotId
+        Preconditions.checkArgument(
+            SnapshotUtil.isParentAncestorOf(table(), toSnapshotIdInclusive, context().fromSnapshotId()),
+            "Starting snapshot (exclusive) %s is not a parent ancestor of end snapshot %s",
+            context().fromSnapshotId(), toSnapshotIdInclusive);
+      }
+    }
+
+    if (fromSnapshotIdExclusive != null) {
+      Listeners.notifyAll(new IncrementalScanEvent(table().name(), fromSnapshotIdExclusive,
+          toSnapshotIdInclusive, context().rowFilter(), table().schema(), false));
+    } else {
+      Snapshot oldestAncestorSnapshot = SnapshotUtil.oldestAncestor(toSnapshotIdInclusive, table()::snapshot);
+      Listeners.notifyAll(new IncrementalScanEvent(table().name(), oldestAncestorSnapshot.snapshotId(),
+          toSnapshotIdInclusive, context().rowFilter(), table().schema(), true));
+    }
+
+    // appendsBetween handles null fromSnapshotId (exclusive) properly
+    List<Snapshot> snapshots = appendsBetween(table(), fromSnapshotIdExclusive, toSnapshotIdInclusive);
+    if (snapshots.isEmpty()) {
+      return CloseableIterable.empty();
+    }
+
+    return appendFilesFromSnapshots(snapshots);
+  }
+
+  @Override
+  public CloseableIterable<CombinedScanTask> planTasks() {
+    CloseableIterable<FileScanTask> fileScanTasks = planFiles();
+    CloseableIterable<FileScanTask> splitFiles = TableScanUtil.splitFiles(fileScanTasks, targetSplitSize());
+    return TableScanUtil.planTasks(splitFiles, targetSplitSize(), splitLookback(), splitOpenFileCost());
+  }
+
+  private CloseableIterable<FileScanTask> appendFilesFromSnapshots(List<Snapshot> snapshots) {
+    Set<Long> snapshotIds = Sets.newHashSet(Iterables.transform(snapshots, Snapshot::snapshotId));
+    Set<ManifestFile> manifests = FluentIterable
+        .from(snapshots)
+        .transformAndConcat(Snapshot::dataManifests)
+        .filter(manifestFile -> snapshotIds.contains(manifestFile.snapshotId()))
+        .toSet();
+
+    ManifestGroup manifestGroup = new ManifestGroup(tableOps().io(), manifests)
+        .caseSensitive(context().caseSensitive())
+        .select(context().returnColumnStats() ? DataTableScan.SCAN_WITH_STATS_COLUMNS : DataTableScan.SCAN_COLUMNS)

Review Comment:
   This block also accesses a lot of information on `context()`. There were some helper methods in `BaseTableScan`. What about moving them to `BaseScan` and using here?
   
   ```
     protected Long snapshotId() {
       return context().snapshotId();
     }
   
     protected boolean colStats() {
       return context().returnColumnStats();
     }
   
     protected boolean shouldIgnoreResiduals() {
       return context().ignoreResiduals();
     }
   
     @Override
     public Expression filter() {
       return context().rowFilter();
     }
   
     @Override
     public boolean isCaseSensitive() {
       return context().caseSensitive();
     }
   
     protected ExecutorService planExecutor() {
       return context().planExecutor();
     }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org