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/25 15:51:09 UTC

[GitHub] [iceberg] aokolnychyi opened a new pull request, #4870: API: Add a scan for changes

aokolnychyi opened a new pull request, #4870:
URL: https://github.com/apache/iceberg/pull/4870

   This PR is an attempt to support a scan for CDC based on @flyrain's proposal and PR.


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623751


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   Added examples to all task types.



##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends ScanTask {
+  /**
+   * Returns the operation type of records produced by this task (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative change order in which the changes must be applied.
+   */
+  int changeOrder();

Review Comment:
   Changed.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911421538


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   @aokolnychyi, I'd say that's a special form of combining. We could introduce a `CombinedDeletedRowsScanTask` that takes care of the snapshot handling.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909082179


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends ScanTask {
+  /**
+   * Returns the operation type of records produced by this task (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative change order in which the changes must be applied.
+   */
+  int changeOrder();

Review Comment:
   What about `changeOrdinal`, since this is an ordinal number?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908673889


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   I updated the Javadoc. If scanning for changes across snapshots, we may have equality deletes too.



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


[GitHub] [iceberg] hameizi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
hameizi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r903406610


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   > It has nothing to do with primary key(s). Iceberg allows two or multiple rows to be identical. For example, here are two rows with the same values in each columns(id, name, age).
   > 
   > ```
   > r1 (1, "Alice", 20) at pos 0 of data file df1
   > r2 (1, "Alice", 20) at pos 1 of data file df1
   > ```
   
   For V2 format should not exist two same data but no delete data. It's invalid.
   
   > 
   > The user won't be able to distinguish the following two scenarios.
   > 
   > 1. Both s1 and s2 deleted r1.
   > 2. S1 deleted r1, while s2 deleted r2. Or vise-verse.
   
   Base on the above this case is invalid. Even this happen, for equal-delete there should be `Both s1 and s2 deleted r1.`, for postion-delete there is specific data what locate at what file and postion should be delete.
   



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881867159


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();

Review Comment:
   this makes sense to me. original `planFiles` is not flexible enough to cover the more complex changelog scan



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889241797


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);

Review Comment:
   Resolving as no longer applies.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889737047


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   We can extract an `IncrementalScan` parent interface so that `IncrementalAppendScan` and `IncrementalChangelogScan` can extends from. Right now, we don't anticipate any methods specific to append-only scans. 
   
   We should still defines/keeps the `IncrementalAppendScan` even if it is identical to `IncrementalScan` (at least for now). `Table#newIncrementalAppendScan` should return `IncrementalAppendScan`. In the future, `Table#newIncrementalChangelogScan` can return the `IncrementalChangelogScan` defined in this PR.
   



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890447623


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Yea I agree, the third option makes sense, but it is a lot of refactoring, and it may break external users of Scan.planFiles()?  I'm also not sure if it help today's problem of trying to re-use a lot of the reader/split logic done today with FileScanTask, unless we extract some logic there to some new base ScanTask type.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r891688733


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   @rdblue, no. I am talking about the same row being deleted multiple times. That can happen both with position or equality deletes. 
   
   Case 1: we have two concurrent merge-on-read DELETEs in Spark. If the operations remove the same row, we allow both commits to succeed. The correct behavior would be to report the matching row as deleted only in the first snapshot as the second snapshot added a delete for an already deleted row.
   
   Case 2: we have two snapshots that both add equality deletes for data file A. When we output changes for the second snapshot, the correct behavior would be to output only the deleted records that were still live (i.e. apply the equality delete added in the first snapshot before checking what was removed in the second snapshot).



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895070981


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   Let's say we delete a row r1 in s1, and delete it again in s2. We will get two CDC delete records, one from s1, another from s2, as you pointed out. The second CDC delete record could be considered as a no-op, which is probably fine(I will come back with an edge case). The problem happens when we only generate CDC records from s2. How do we handle the deleted row r1 in s2? Do we output it or not? It is confusing if we output it since it is really a no-op.
   
   The edge case is that there are two same rows in the data files(r1, r2). We don't know whether r1 is deleted twice or both r1 and r2 are deleted if we output duplicated delete cdc records. Unless we project row pos and data file name in cdc records, which sounds weird, and exposes two much internal implementation details.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r894745659


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   Multi-snapshot support is yet to be decided, and even with that, we can have this interface extend the one for single snapshot.



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


[GitHub] [iceberg] hameizi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
hameizi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r903393610


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   > > The problem happens when we only generate CDC records from s2
   > 
   > I feel it is an invalid case. CDC read should been incremental and continuous. If an Iceberg table is the CDC source, I would imagine CDC read should start with full table scan, then transition to the incremental changelog scan. We shouldn't read S2 without reading S1. Even if CDC read started from S2 and continues forward, it is also ok to emit a delete in my mind.
   
   +1



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909079955


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   I may not be completely following, so correct me if I'm not understanding something.
   
   This interface appears to be written to contain changes from a single snapshot. For example, it could be reading deletes that were committed via a delete file that was added in snapshot `S`. In that case, `operation=DELETE`, `changeOrder=ord(S)`, and `commitSnapshotId=S`.
   
   If I understand this thread, the question is how to handle "net changes across snapshots", which I interpret as basically "squashing" snapshots to produce the output. If `S1` adds rows with ids 4, 5, and 6, then `S2` deletes the rows with id 5 and 7, then the result could be the rows with ids 4 and 6, along with a delete for id 7.
   
   We can avoid the problem by always producing one task for `S1` with rows 4, 5, and 6, and then another task (or set of tasks) for `S2` with the delete for ids 5 and 7, which is what the interface appears to be written for. But I think the point is that we can read data files normally and apply deletes to easily produce the squashed version, which could be quite valuable to end users.
   
   I think we can solve this by setting the commit snapshot and change order to the values for `S1`. The difference between the two cases is that the read of the inserted data applies the newer deletes in the window of snapshots. So that task will always produce values that were added in snapshot `S1`, rows 4 and 5. Then a separate task would be created for the deletes from `S2` (with `S2` for its `commitSnapshotId`), which would not emit id 5 because its change order was not strictly less than `ord(S1)`. That task would produce a deleted row for id 7, using the delete task's snapshot ID. So I think in all cases this works. We just still need to separate the deletes in their own task, still.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909084592


##########
api/src/main/java/org/apache/iceberg/Table.java:
##########
@@ -64,6 +64,17 @@ default IncrementalAppendScan newIncrementalAppendScan() {
     throw new UnsupportedOperationException("Incremental append scan is not supported");
   }
 
+  /**
+   * Create a new {@link IncrementalChangelogScan} for this table.
+   * <p>
+   * Once a scan is created, it can be refined to project columns and filter data.
+   *
+   * @return an incremental changelog scan
+   */
+  default IncrementalChangelogScan newIncrementalChangelogScan() {

Review Comment:
   Do we want `changelog` or simply `change`?



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909111695


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   > I think the point is that we can read data files normally and apply deletes to easily produce the squashed version, which could be quite valuable to end users.
   
   Yes. That's one of the use case we think of, which could be resolved by separating the tasks. 
   
   Here is another one, which is a bit different from the above use case, `S1` adds rows with ids 4, 5, and 6, then `S2` deletes the rows with id 5 and 7, `S2` also adds the rows with id 8, 9. We want to have a single `AddedRowsScanTask` to get all added rows with s1 and s2. To give one snapshot id doesn't make sense since added rows 4, 6, 8, 9 come from different snapshots. Of course, we archive the same result by splitting it into two `AddedRowsScanTask`s, and merge them later. But a task across multiple multiple snapshots may have perf advantage, or simpler implementation for that use case. What do you think?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890415034


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   That may be reasonable cause we can return `ChangelogScanTask` from this API but I think all other questions still remain. We will have to adapt our readers and refactor the split logic, etc. The main question is whether we consider that worth the effort as technically we can reuse `FileScanTask` for all types of work.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888474315


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   Are we restrict this scan task for one snapshot? I feel like itself and its subclass can be used for a scan across multiple snapshots, which is useful, for example, to scan position deletes across snapshots. In that case, commitSnapshotId is going to be optional.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887255666


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();

Review Comment:
   I reconsidered this point as I am thinking of adapting the readers instead.



##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {

Review Comment:
   Fixed.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888238729


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Follow the discussing on https://github.com/apache/iceberg/pull/4812#discussion_r887384420, there's a discussion on whether we can and should refactor FileScanTask.  This is today's situation, baked into the code and used in many places:
   
   1. Scan returns FileScanTask
   2. FileScanTask has methods DataFile file() and List<DeleteFile> deleteFiles(), which makes sense only in case of scanning data files.
   3. BaseFileScanTask has method Iterable<FileScanTask> split(), which uses file() to calculate the splits.
   
   Problem: for growing number of cases , like CDC, read position deletes, writing a new scan implementation forces FileScanTask to be returned, which will then have to unnaturally wrap non-data files as DataFile file().  In these cases, even wrapping a DeleteFile as a DataFile.
   
   I see two options
   
   Option 1:  Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc.  Minimal code change, but messy.  We will see DeleteDataFile.
   
   Option 2:  Refactor FileScanTask hierarchy.
   One potential solution is in #4812:
   
   1. Scan still returns FileScanTask (cant change this)
   2. FileScanTask now becomes a parameterized base class with method : ContentFile<T> contentFile()
   3. BaseFileScanTask will do all the splitting logic based on contentFile().  In fact, this is a small code replacement of using file() => contentFile() as already today all the methods it uses are in fact from contentFile and not specific to DataFile.
   4. Eventually, we make FileScanTask subtype DataFileScanTask with methods DataFile dataFile() and List<DeleteFile> deleteFiles(), and code that deals exclusively with data files should use this scan task for clarity. 
   5. Deprecate current FileScanTask file() and deleteFiles() and eventually remove them in favor of the subclass (though with 1.0, we may need to wait for next major relesae to do this).
   
   With this solution, we need only minimal changes for 1-3, and can do more refactoring of 4-5 in different prs gradually.  It also preserves backward compatibility.  I think this solution would be clearer in the code, where we are not dealing exclusively with DataFile but rather different content files.
   
   But I am not entirely sure what kind of interface CDC FileScanTask will need, so will be open to see if this works or what you or others have thought.
   
   Also, let me know if this is the not the place you meant for this discussion. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911440819


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   We won't be able to extend `ChangelogScanTask` in `CombinedDeletedRowsScanTask`, though? Unless we remove the commit snapshot ID from `ChangelogScanTask`?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881906894


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {

Review Comment:
   I thought about this but I consider `Changelog` as something that already contains the records (e.g. the result of scanning these files). This felt more like an index where you would look files up.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893940958


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I think we generally want to avoid DataTask



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881953361


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   If we resolve equality deletes, then we don't have to differentiate between equality or position deletes.
   We can simply use `is_deleted` metadata column for both.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887272758


##########
api/src/main/java/org/apache/iceberg/AddedDataFileScanTask.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that matching position deletes added in the same snapshot must be applied while reading the data file.
+ */
+public interface AddedDataFileScanTask extends ChangelogScanTask {

Review Comment:
   Naming ideas are welcome. I am still iterating on these.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888485972


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Option 2 looks more reasonable. With that, it makes sense to extend FileScanTask from ChangelogScanTask, which reduce the place we need to change. 



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889215815


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   what is the scope of the order? is it  the snapshot sequence number? or is it just a relative order for all tasks from one plan result?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889231846


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   Snapshot sequence numbers are present only in v2 tables. Also, some records may have different commit orders within one snapshot. Consider a Flink upsert that adds a data file and a position delete file against it. Added records should have a smaller commit order.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890401461


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   There may be a third option. What about parameterizing `Scan` so that it has a task type? Then `DataTableScan` could still use `FileScanTask`, but we could introduce more task types that are different. Then we don't need to change existing scan classes or tasks, but we'd be able to reuse a lot more of the code.
   
   Do you think that would work?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889737047


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   We can extract an `IncrementalScan` parent interface so that `IncrementalAppendScan` and `IncrementalChangelogScan` can extends from. Right now, we don't anticipate any methods specific to append-only scans. We should still defines/keeps the `IncrementalAppendScan` which is identical to `IncrementalScan` (at least for now). 
   
   `Table#newIncrementalAppendScan` should return `IncrementalAppendScan`. In the future, `Table#newIncrementalChangelogScan` can return the `IncrementalChangelogScan` defined in this PR.
   



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889736476


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   commit implies a single snapshot/transaction to me. what about `changeOrder`?
   ```
   Returns the relative order in which the changes must be applied.
   ```



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893833783


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   Eq delete doesn't apply to the data files in the same snapshot, which is written in the spec, https://iceberg.apache.org/spec/#scan-planning.
   
   > An equality delete file must be applied to a data file when all of the following are true:
   > The data file’s sequence number is strictly less than the delete’s sequence number



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893929732


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   If we output pos and data file names in the result, we can understand the rows being deleted multiple times, and deduplicate them while merging. In that way, we don't have to change anything in the scan task. Oh, that doesn't solve the case that we don't merge snapshots which deletes the same row multiple times. Never mind, it won't work for all use cases. 



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r912531270


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   I think either interpretation of the concurrent delete case is fine. For the position delete example, d1 and d2 are concurrent and based on the same underlying data, so I'm fine saying BOTH of them deleted a row. It is a little strange to say d1 deleted pos 0 and d2 didn't because d2 actually did encode that delete and would have deleted row 0 if it had won the race to commit. But there is also a strong argument that we should produce a delete for each row once to avoid confusion when people consume these tables.
   
   I think as long as the implementation documents what it does, either one is fine.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909100489


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   @aokolnychyi, how about creating a new class instead of a `Map<String, String>` for `commitMetadata`? The `commitMetadata` will have `int commitOrder`, `long snapshotId`, `long timestamp`. In case that we have something else needed to pass to reader through the task, we can change the class `commitMetadata` instead of changing this interface.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889236453


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   When we scan for changes across snapshots, I think we should output all changes per snapshot by default. If I understand you correctly, you want to support use cases when the user is only interested in net changes across snapshots. I think that's possible too but the records produced by a task would still have some snapshot ID when they were committed.
   
   For instance, we add data file A in S1. Then we add a delete file D in S2 that matches data file A. In this case, we would output `AddedDataFileScanTask` with commit snapshot S1 but it will have a delete file D associated with it so that we will only output records that remain there in S2.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889240970


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);
+
+    /**
+     * Returns scan tasks for equality delete files that were not assigned to matching data files in a given snapshot.
+     */
+    Iterable<FileScanTask> unresolvedEqualityDeleteFiles(long snapshotId);

Review Comment:
   Resolving as no longer applies.



##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   Resolving as no longer applies.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889239780


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   @stevenzwu, was the plan to not have  `IncrementalAppendScan ` at all and return `IncrementalScan` from `newIncrementalAppendScan` in `Table`? Do we anticipate any methods specific to append-only scans? Like whether to ignore overwrite snapshots or fail if an overwrite is present?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888550825


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public enum ChangelogOperation {

Review Comment:
   Yeah, I googled and seems to come up a lot as a single word. I originally thought those are two words.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889128713


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public enum ChangelogOperation {

Review Comment:
   yeah. I think single word of `Changelog` is probably more common



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910421151


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   I think a quick example in each of these java docs would be helpful.  Like suggested by @rdblue above, just what is the expected output of a given task type.
   
   For example here
   ```
   A scan task for producing records which were deleted because new delete files were added to the table.
   
   Given File A with rows (1 - 10) IN S1 with row 5 marked as deleted by a delete file D1.
   Given S2 with a new Delete File D2 deleting row 6
   
   DRowScanTask (S2) returns {
   files : (a)
   addedDeletes: (D2)
   existingDeletes: (D1)
   }
   Which should return row 6 because that row is newly marked as deleted by S2
   ```
   Or something like that



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911441868


##########
api/src/main/java/org/apache/iceberg/Table.java:
##########
@@ -64,6 +64,17 @@ default IncrementalAppendScan newIncrementalAppendScan() {
     throw new UnsupportedOperationException("Incremental append scan is not supported");
   }
 
+  /**
+   * Create a new {@link IncrementalChangelogScan} for this table.
+   * <p>
+   * Once a scan is created, it can be refined to project columns and filter data.
+   *
+   * @return an incremental changelog scan
+   */
+  default IncrementalChangelogScan newIncrementalChangelogScan() {

Review Comment:
   @stevenzwu, how do you feel about the names? You know more about CDC use cases. I think I'd slightly prefer shorter names like `ChangeScan` and `ChangeScanTask` but if `changelog` is really widely used, I have no problem using it as well.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908700027


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   We can make this configurable if we want as I don't see this being an issue if Flink CDC is the only writer.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887373186


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Another open question is whether we should extend `DataTask`. We need to add extra columns with metadata such as commit order and operation. I decided not to do that to support vectorization in the future.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890314295


##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   I wanted to handle cases when we should output the content of equality deletes as-is. That said, I'll probably skip this for now to simplify the initial implementation and we can reconsider it back.



##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   I wanted to handle cases when we should output the content of equality deletes as-is. That said, I'll probably skip this for now to simplify the initial implementation and we can reconsider it later.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881947028


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   I think we need another method that returns the data files with pos deletes file only, since it is an option to handle eq delete separately.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895346775


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   It has nothing to do with primary key(s). Iceberg allows two or multiple rows to be identical. For example,  here are two rows with the same values in each columns(id, name, age).
   ```
   r1 (1, "Alice", 20) at pos 0 of data file df1
   r2 (1, "Alice", 20) at pos 1 of data file df1
   ```
   Assume both s1 and s2 delete r1,  then we will emit two delete records.
   ```
   (1, "Alice", 20, "D", "S1")
   (1, "Alice", 20, "D", "S2")
   ```
   The user won't be able to distinguish the following two scenarios. 
   1. Both s1 and s2 deleted r1.
   2. S1 deleted r1, while s2 deleted r2. Or vise-verse.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888242476


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   fyi @RussellSpitzer @flyrain @stevenzwu 



##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   fyi @RussellSpitzer @flyrain @stevenzwu  for any thoughts



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889623767


##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   why is this interface needed in addition to `DeletedRowsScanTask`? If this is for equality deletes, does it mean `DeletedRowsScanTask` is only for position deletes?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889623615


##########
api/src/main/java/org/apache/iceberg/AddedDataFileScanTask.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that matching position deletes added in the same snapshot must be applied while reading the data file.
+ */
+public interface AddedDataFileScanTask extends ChangelogScanTask {

Review Comment:
   maybe `InsertedRowsScanTask` similar to `DeletedRowsScanTask`?



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888474315


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   Do we restrict this scan task for one snapshot? I feel like itself and its subclass can be used for a scan across multiple snapshots, which is useful, for example, to scan position deletes across snapshots. In that case, commitSnapshotId is going to be optional.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r891621391


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   Are you talking about position deletes within the same snapshot?
   
   I thought for that we were going to check the sequence number and adjust behavior. So there's no need for a flag.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r894014725


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   Think a bit more. We have 2 options.
   Option 1. Reader side change, based on what Anton proposed.
       1. Need changes for deletes() in FileScanTask to return a list of historical delete files as well as current snapshot's delete files.
       2. Output only if rows are deleted in the current deletes are not in the previous ones.
       3. Need changes from the readers. Actually it needs a special reader only for this purpose, which return the deleted rows in this snapshot but not in the previous snapshots. This part of logic cannot be reused by `_deleted` metadata column. 
   
   For example, a change like this:
   ```
     public static <T> CloseableIterable<T> markDeleted(CloseableIterable<T> rows, Predicate<T> isDeleted,
                                                        Consumer<T> deleteMarker) {
       return CloseableIterable.transform(rows, row -> {
         if (isDeleted.test(row) && isNotInHistoricalDelete.test(row)) {
           deleteMarker.accept(row);
         }
         return row;
       });
     }
   ```
   
   Option 2. Use the Spark dataframe to deduplicate
       1. Read previous deleted rows into a dataframe df1, read the current deleted rows into a datadrame df2
       2. `df2.exceptAll(df1)` to deduplicate
       3. we need a new scan task for previous deletes, maybe just a minor change of the current DeletedRowsScanTask. For example, for data file f1, s1 added delete file d1, and s2 added delete file d2, and s3 added delete file d3, assume s3 is the current snapshot the scan task for previous deletes should has f1, d1, d2.
       4. We need to output pos and data file name to identify the duplication.
       
   Option 1 needs changes from reader, but it should performance better than option 2. Option 2 need less code change for scan task and reader, but it may not perform well, and cannot be used across engines.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893929732


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   If we output pos and data file names in the result, we can understand the rows being deleted multiple times, and deduplicate them while merging. In that way, we don't have to change anything in the scan task. Oh, that doesn't work for the case that we don't merge snapshots which deletes the same row multiple times. Never mind.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893929732


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   If we output pos and data file names in the result, we can understand the rows being deleted multiple times, and deduplicate them while merging. In that way, we don't have to change anything in the scan task.



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


[GitHub] [iceberg] chenjunjiedada commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r894069488


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   Does this assume it only scans one snapshot? From the comment here:https://github.com/apache/iceberg/pull/4870/files#r889236453, it looks like support scan across snapshots? 



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


[GitHub] [iceberg] aokolnychyi commented on pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#issuecomment-1144104946

   I spent some time thinking and decided that separate scan task types would work a little bit better. Then we can adapt our readers to handle different task types independently whether we will use a utility or a metadata table scan approach.
   
   


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


[GitHub] [iceberg] chenjunjiedada commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
chenjunjiedada commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r891872580


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   Do we need to care about the equality deletes as well?



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909100489


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   @aokolnychyi, how about creating a new class instead of a `Map<String, String>` for `commitMetadata`? The `commitMetadata` will have `int commitOrder`, `long snapshotId`, `long timestamp`. In case, we have something else needed to pass to reader through the task, we can change the class `commitMetadata` instead of changing this interface.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909081267


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.

Review Comment:
   Should these tasks have a description of what rows should be produced by readers?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623552


##########
api/src/main/java/org/apache/iceberg/IncrementalScan.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+/**
+ * API for configuring an incremental scan.
+ */
+public interface IncrementalScan<ThisT, T extends ScanTask, G extends ScanTaskGroup<T>> extends Scan<ThisT, T, G> {
+  /**
+   * Refines this scan with a start snapshot ID (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   * @throws IllegalArgumentException if the start snapshot is not an ancestor of the end snapshot
+   */
+  ThisT fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Refines this scan with a start snapshot ID (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   * @throws IllegalArgumentException if the start snapshot is not an ancestor of the end snapshot
+   */
+  ThisT fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).

Review Comment:
   I aligned methods above.



##########
api/src/main/java/org/apache/iceberg/IncrementalScan.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+/**
+ * API for configuring an incremental scan.
+ */
+public interface IncrementalScan<ThisT, T extends ScanTask, G extends ScanTaskGroup<T>> extends Scan<ThisT, T, G> {
+  /**
+   * Refines this scan with a start snapshot ID (inclusive).

Review Comment:
   Updated.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881866293


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {

Review Comment:
   what about `Changelog`?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881830930


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {

Review Comment:
   nit: what about `IncrementalChangelogScan`?
   
   Also it probably should be `extends Scan< ChangeScan>`



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


[GitHub] [iceberg] rdblue merged pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue merged PR #4870:
URL: https://github.com/apache/iceberg/pull/4870


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889243787


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   The idea is that the commit order will start from 0 every time we query for a set of changes and will indicate in which order changes should be applied.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890418254


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   I added `IncrementalScan`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908673100


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   This question is still open. I mainly consider 2 options to handle scans for net changes across snapshots:
   
   - Switch to `Map<String, String> commitMetadata()` that will contain things like snapshot ID and commit timestamp. That would mean having a map instead of separate columns like `_commit_snapshot_id`. It will be then easier to add more info later but it does make it harder to access what's stored in the map.
   - Switch to `Long` from `long` and make it null if we are scanning across snapshots.
   
   I'd be curious to hear what other folks think.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908666853


##########
api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java:
##########
@@ -20,6 +20,8 @@
 
 package org.apache.iceberg;
 
+import org.apache.iceberg.io.CloseableIterable;
+
 /**
  * API for configuring an incremental table scan for appends only snapshots

Review Comment:
   I added a parent interface. Resolving this.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911441868


##########
api/src/main/java/org/apache/iceberg/Table.java:
##########
@@ -64,6 +64,17 @@ default IncrementalAppendScan newIncrementalAppendScan() {
     throw new UnsupportedOperationException("Incremental append scan is not supported");
   }
 
+  /**
+   * Create a new {@link IncrementalChangelogScan} for this table.
+   * <p>
+   * Once a scan is created, it can be refined to project columns and filter data.
+   *
+   * @return an incremental changelog scan
+   */
+  default IncrementalChangelogScan newIncrementalChangelogScan() {

Review Comment:
   @stevenzwu, how do you feel about the names? You know more about this CDC use cases. I think I'd slightly prefer shorter names like `ChangeScan` and `ChangeScanTask` but if `changelog` is really widely used, I have no problem using it as well.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910313539


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   Sounds good to me.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r902844831


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   It's OK to leave `commitSnapshotId` and `commitOrder` null in case of net changes. I assume the intent to have method `commitSnapshotId` and `commitOrder` is to enable reader project them. 
   <meta charset="utf-8"><b style="font-weight:normal;" id="docs-internal-guid-f812f805-7fff-9766-1a99-7a96c16e3f5e"><div dir="ltr" style="margin-left:0pt;" align="left">
   
   ID | Name | Age | _record_type | _commit_snapshot_id | _commit_timestamp | _commit_order
   -- | -- | -- | -- | -- | -- | --
   1 | Amy | 21 | Insert | 8563478696506531065 | 2021-01-23 04:30:45 | 34
   
   </div></b>
   Considering the cdc record format above, does reader need `commitTimestamp` as well? 
   
   



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r882091710


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {

Review Comment:
   agree with the implication of `Changelog`.  it is not appropriate here. However I also feel `index` is not the most intuitive name here. Is `PlanResult` informative?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881831893


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {

Review Comment:
   Yeah, that's definitely a typo! The name sounds good to me too.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881827887


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();

Review Comment:
   The original `planFiles` did not work well as we have different types of files to plan and those will have to be read and processed differently. Any other ideas are welcome!



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887276736


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I am debating whether we should extend `FileScanTask` or not.
   
   Some benefits of extending:
   - Our usage kind of fits the description: `A scan task over a range of a single file`.
   - We can reuse split logic and `CombinedScanTask`.
   - We can reuse the existing `planFiles` and `planTasks` methods.
   - We can reuse a lot of reader code that assumes we get some sort of `FileScanTask`. 



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888238729


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Follow the discussing on https://github.com/apache/iceberg/pull/4812#discussion_r887384420, there's a discussion on whether we can and should refactor FileScanTask.  Today, this is baked into the code and used heavily:
   
   1. Scan returns FileScanTask
   2. FileScanTask has methods DataFile file() and List<DeleteFile> deleteFiles(), which makes sense only in case of scanning data files.
   3. BaseFileScanTask has method Iterable<FileScanTask> split(), which uses file() to calculate the splits.
   
   Problem: for growing number of cases , like CDC, read position deletes, writing a new scan implementation forces FileScanTask to be returned, which will then have to unnaturally wrap non-data files as DataFile file().  In these cases, even wrapping a DeleteFile as a DataFile.
   
   I see two options
   
   Option 1:  Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc.  Minimal code change, but messy.  We will see DeleteDataFile.
   
   Option 2:  Refactor FileScanTask hierarchy.
   One potential solution is in #4812:
   
   1. Scan still returns FileScanTask (cant change this)
   2. FileScanTask now becomes a parameterized base class with method : ContentFile<T> contentFile()
   3. BaseFileScanTask will do all the splitting logic based on contentFile().  In fact, this is a small code replacement of using file() => contentFile() as already today all the methods it uses are in fact from contentFile and not specific to DataFile.
   4. Eventually, we make DataFileScanTask that has methods DataFile dataFile() and List<DeleteFile> deleteFiles(), and code that deals exclusively with DataFileScanTask can use that for clarity. 
   5. Deprecate current FileScanTask file() and deleteFiles() and eventually remove them in favor of the subclass (though with 1.0, not sure the timeline we can do this).
   
   With this solution, we need only minimal changes for 1-3, and can do more refactoring of 4 in different prs, while preserving backward compatibility.  I like this solution as it keeps some kind of clarity in the code that we are not dealing with DataFile but rather different content files.
   
   But I am not entirely sure what kind of interface CDC FileScanTask will need, so will be open to see if this works.
   
   Also, let me know if this is the not the place you meant for this discussion. 



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r882096083


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   would the caller handle the read of this case and the `addedDataFiles` differently? or maybe ask in a different way, are the returned data files here from the same snapshot?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r882096083


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   would the caller handle the read of this case and the `addedDataFiles` differently?



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881960111


##########
api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java:
##########
@@ -20,6 +20,8 @@
 
 package org.apache.iceberg;
 
+import org.apache.iceberg.io.CloseableIterable;
+
 /**
  * API for configuring an incremental table scan for appends only snapshots

Review Comment:
   Logically, incrementalAppendScan is a subset of ChangeScan. Can we make incrementalAppendScan extend from ChangeScan?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888182279


##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   This task is a little bit weird as it would wrap an equality delete file as `DataFile`. We do that somewhere else but it may be a good precedent to reconsider extending `FileScanTask`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889240594


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {

Review Comment:
   Resolving as it no longer applies.



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


[GitHub] [iceberg] aokolnychyi commented on pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#issuecomment-1147748743

   I've simplified this PR to always resolve equality deletes for now. I feel like it is easier to get this working first add the option of not resolving equality deletes later. Right now, all added scan tasks fit well into `FileScanTask`.


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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890790769


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   I've discovered one issue implementing this API locally. Specifically, we need to differentiate between historical deletes and new deletes in this task. If we add a new delete file, we should NOT output records that were deleted in earlier snapshots. We need to apply old deletes first and then see what was matched with the new deletes. It's not something we can express with `FileScanTask`. I could add another method for new deletes and return deletes from earlier snapshots in the existing `deletes()` method. That sounds awkward.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890447623


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Yea I agree, the third option makes sense, but it is a lot of refactoring (not to mention having to migrate external users).  I'm also not sure if it help today's problem of trying to re-use a lot of the reader/split logic done today with FileScanTask, unless we extract some logic there to some new base ScanTask type.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893833783


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   Eq delete doesn't apply to the data files in the same snapshot, which is written in the spec, https://iceberg.apache.org/spec/#scan-planning.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908697632


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   I want to give specific examples for cases I mentioned [earlier](https://github.com/apache/iceberg/pull/4870#discussion_r891688733).
   
   **Concurrent merge-on-read DELETEs in Spark**
   
   We have `data_file_A`:
   
   ```
   1, "a" (pos 0)
   2, "b" (pos 1)
   3, "c" (pos 2)
   4, "d" (pos 3)
   ```
   
   Suppose we have two concurrent DELETEs (`d1` and `d2`). The first DELETE removes records in `pos 0` and `pos 2`. The second one concurrently removes `pos 0` and `pos 1`. We allow the second DELETE to commit cause it is not in conflict.
   
   How should our changelog look like?
   
   Changelog for `d1`:
   
   ```
   deleted, 1, "a"
   deleted, 3, "c"
   ```
   
   Changelog for `d2`:
   
   ```
   deleted, 2, "b"
   ```
   
   I think `1, "a"` should only appear in `d1` despite that a delete file added in `d2` refers to it.
   
   **Equality deletes against the same data file**
   
   We have `data_file_A`:
   
   ```
   1, "hr" (pos 0)
   2, "sw" (pos 1)
   3, "hr" (pos 2)
   4, "sw" (pos 3)
   ```
   
   Suppose we have a GDPR delete `d1` that adds an equality delete for `1` and a concurrent equality delete `d2` that removes all records in `hr` department.
   
   How should our changelog look like?
   
   Changelog for `d1`:
   
   ```
   deleted, 1, "hr"
   ```
   
   Changelog for `d2`:
   
   ```
   deleted, 3, "hr"
   ```
   
   I don't think outputting `1, "hr"` again in `d2` would be correct as that record wasn't live when `d2` committed.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910421151


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   I think a quick example in each of these java docs would be helpful.  Like suggested by @rdblue above, just what is the expected output of a given task type.
   
   For example here
   ```
   A scan task for producing records which were deleted because new delete files were added to the table.
   
   Given File A with rows (1 - 10) IN S1 with row 5 marked as deleted by a delete file D1.
   Given S2 with a new Delete File D2 deleting rows >= 4
   
   DRowScanTask (S2) returns {
   files : (a)
   addedDeletes: (D2)
   existingDeletes: (D1)
   }
   Which should return row 4, 6, 7, 8, 9, 10  because those rows are newly marked as deleted by S2 while row 5 was already deleted by D1 in S1.
   ```
   Or something like that



##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   I think a quick example in each of these java docs would be helpful.  Like suggested by @rdblue above, just what is the expected output of a given task type.
   
   For example here
   ```
   A scan task for producing records which were deleted because new delete files were added to the table.
   
   Given File A with rows (1 - 10) IN S1 with row 5 marked as deleted by a delete file D1.
   Given S2 with a new Delete File D2 deleting rows >= 4
   
   DRowScanTask (S2) returns {
   files : (a)
   addedDeletes: (D2)
   existingDeletes: (D1)
   }
   Which should return row 4, 6, 7, 8, 9, 10  because those rows are newly marked as 
   deleted by S2 while row 5 was already deleted by D1 in S1.
   ```
   Or something like that



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910421151


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   I think a quick example in each of these java docs would be helpful.  Like suggested by @rdblue above, just what is the expected output of a given task type.
   
   For example here
   ```
   A scan task for producing records which were deleted because new 
   delete files were added to the table.
   
   Given File A with rows (1 - 10) IN S1 with row 5 marked as deleted by a delete file D1.
   Given S2 with a new Delete File D2 deleting rows >= 4
   
   DRowScanTask (S2) returns {
   files : (a)
   addedDeletes: (D2)
   existingDeletes: (D1)
   }
   Which should return row 4, 6, 7, 8, 9, 10  because those rows are newly marked as 
   deleted by S2 while row 5 was already deleted by D1 in S1.
   ```
   Or something like that



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911418335


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   @flyrain @rdblue, I agree with you but what about `DeletedRowsScanTask`?
   
   Suppose I have a data file `file_A`:
   
   ```
   id
   --
   1
   2
   3
   4
   5
   ```
   
   And then I have two delete snapshots S1 and S2. S1 deletes IDs 1, 2 and S2 deletes IDs 3, 4. If we are to assign snapshot IDs to tasks, we will have to produce two `DeletedRowsScanTask`s:
   
   ```
   file_A & deletes from S1 (inherit the snapshot ID and commit order from S1)
   file_A & deletes from S2 (inherit the snapshot ID and commit order from S2)
   ```
   
   Even if we combine these two tasks in one group, it will be suboptimal as we will scan through the same data file twice. It would be more efficient to combine these deletes against the same data file and do one pass to determine deleted records.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911518357


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   I see your point, but I'm thinking that we would just throw an exception if it were called. I'd also be up for returning null and passing each sub-task's ID and ordinal separately.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887276736


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I am debating whether we should extend `FileScanTask` or not.
   
   Some benefits of extending:
   - Our usage kind of fits the description: `A scan task over a range of a single file`.
   - We can reuse split logic and `CombinedScanTask`.
   - We can reuse the existing `planFiles` and `planTasks` methods.
   - We can reuse a lot of reader code that assumes we get some sort of `FileScanTask`. This is really useful as we can use `JoinedRow` in Spark to add extra metadata to an iterable of `InternalRows` we get back by reading regular tasks.



##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I am debating whether we should extend `FileScanTask` or not.
   
   Some benefits of extending:
   - Our usage kind of fits the description: `A scan task over a range of a single file`.
   - We can reuse split logic and `CombinedScanTask`.
   - We can reuse the existing `planFiles` and `planTasks` methods.
   - We can reuse a lot of reader code that assumes we get some sort of `FileScanTask`. This is really useful as we can use `JoinedRow` in Spark to add extra metadata to an iterable of `InternalRow`s we get back by reading regular tasks.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887276736


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I am debating whether we should extend `FileScanTask` or not.
   
   Some benefits of extending:
   - Our usage kind of fits the description: `A scan task over a range of a single file`.
   - We can reuse split logic and `CombinedScanTask` (the last one works only with `FileScanTask`).
   - We can reuse the existing `planFiles` and `planTasks` methods.
   - We can reuse a lot of reader code that assumes we get some sort of `FileScanTask`. This is really useful as we can use `JoinedRow` in Spark to add extra metadata to an iterable of `InternalRow`s we get back by reading regular tasks.



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888238729


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Follow the discussing on https://github.com/apache/iceberg/pull/4812#discussion_r887384420, there's a discussion on whether we can and should refactor FileScanTask.  This is today's situation, baked into the code and used in many places:
   
   1. Scan returns FileScanTask
   2. FileScanTask has methods DataFile file() and List<DeleteFile> deleteFiles(), which makes sense only in case of scanning data files.
   3. BaseFileScanTask has method Iterable<FileScanTask> split(), which uses file() to calculate the splits.
   
   Problem: for growing number of cases , like CDC, read position deletes, if we want to write a new scan but re-use split logic and existing integration with all the engines, it forces FileScanTask to be returned, which will then have to unnaturally wrap non-data files as DataFile file().  In these cases, even wrapping a DeleteFile as a DataFile.
   
   I see two options
   
   Option 1:  Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc.  Minimal code change, but messy.  We will see DeleteDataFile.
   
   Option 2:  Refactor FileScanTask hierarchy.
   One potential solution is in #4812:
   
   1. Scan still returns FileScanTask (cant change this)
   2. FileScanTask now becomes a base class with method : ContentFile<T> contentFile()
   3. BaseFileScanTask will do all the splitting logic based on contentFile().  In fact, this is a small code replacement of using file() => contentFile() as already today all the methods it uses are in fact from contentFile and not specific to DataFile.
   4. Eventually, we make FileScanTask subtype DataFileScanTask with methods DataFile dataFile() and List<DeleteFile> deleteFiles(), and code that deals exclusively with data files should use this scan task for clarity. 
   5. Deprecate current FileScanTask file() and deleteFiles() and eventually remove them in favor of the subclass (though with 1.0, we may need to wait for next major relesae to do this).
   
   With this solution, we need only minimal changes for 1-3, and can do more refactoring of 4-5 in different prs gradually.  It also preserves backward compatibility.  I think this solution would be clearer in the code, where we are not dealing exclusively with DataFile but rather different content files.
   
   But I am not entirely sure what kind of interface CDC FileScanTask will need, so will be open to see if this works or what you or others have thought.
   
   Also, let me know if this is the not the place you meant for this discussion. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889243787


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   The idea is that the commit order will start from 0 every time we query for a set of changes.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895053311


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   I am not sure we need to de-dup the delete marker. If `delete from table where key=k1` is called twice, that should be fine. 2nd delete would be no-op.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890417981


##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   I removed this task type for now.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890417440


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   Updated.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r891552574


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   @aokolnychyi is it necessary to suppress the new deletes? We got new deletes because some writer deleted the same row again.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889623767


##########
api/src/main/java/org/apache/iceberg/DeleteMarkersScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for delete markers generated by adding equality delete files to the table.
+ * <p>
+ * Note that such tasks are created only if equality deletes are not assigned to data files
+ * and are used purely for producing delete markers.
+ */
+public interface DeleteMarkersScanTask extends ChangelogScanTask {

Review Comment:
   why is this interface needed in addition to `DeletedRowsScanTask`? 



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910412896


##########
api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by removing a data file from the table.
+ * <p>
+ * Note that all historical delete files added before must be applied while reading the data file to
+ * output only those data records that were live when the data file was removed.
+ */
+public interface DeletedDataFileScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of {@link DeleteFile delete files} to apply when reading the task's data file.
+   *
+   * @return a list of delete files to apply
+   */
+  List<DeleteFile> deletes();

Review Comment:
   I feel like this is related to our "existingDeletes" in the DeletedRowsScan task than it is to `deletes` in `AddedRowScanTask`. Maybe worth changing the name?
   
   Just trying to differentiate between files marking rows which have been deleted so we aren't rendering them and files marking rows which were deleted in the past so we aren't rendering them because we are looking for current deletes.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r902846178


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,33 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching position deletes produced in the same snapshot.

Review Comment:
   We need to consider equality deletes as well if this task can be used for multiple snapshots(net change use case).



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909082030


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching delete files. If so, such delete files will be included

Review Comment:
   Could you clarify when there will be matching delete files? I believe that the intent is to include such delete files when squashing changes.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911625184


##########
api/src/main/java/org/apache/iceberg/Table.java:
##########
@@ -64,6 +64,17 @@ default IncrementalAppendScan newIncrementalAppendScan() {
     throw new UnsupportedOperationException("Incremental append scan is not supported");
   }
 
+  /**
+   * Create a new {@link IncrementalChangelogScan} for this table.
+   * <p>
+   * Once a scan is created, it can be refined to project columns and filter data.
+   *
+   * @return an incremental changelog scan
+   */
+  default IncrementalChangelogScan newIncrementalChangelogScan() {

Review Comment:
   Seems `changelog` is pretty popular. Let's keep it as-is.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881949205


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);
+
+    /**
+     * Returns scan tasks for equality delete files that were not assigned to matching data files in a given snapshot.
+     */
+    Iterable<FileScanTask> unresolvedEqualityDeleteFiles(long snapshotId);

Review Comment:
   The idea is that this method would only return equality deletes not assigned to data files (if `resolveEqualityDeletes` is not called). If we assign equality deletes, then this will be empty.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890426716


##########
api/src/main/java/org/apache/iceberg/AddedDataFileScanTask.java:
##########
@@ -0,0 +1,32 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that matching position deletes added in the same snapshot must be applied while reading the data file.
+ */
+public interface AddedDataFileScanTask extends ChangelogScanTask {

Review Comment:
   Renamed to `AddedRowsScanTask`.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890423544


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I took a quick look and refactoring seems to require quite some work. I am not yet convinced it would be worth the effort but if folks support it, I'll help reviewing.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890423544


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   I took a quick look and refactoring seems to require quite some work.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893871912


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   What will `commitSnapshotId()` return in that case? s1 or s2, or a list of snapshot ids? I'd think the snapshot ids may not be useful if we want to calculate the net changes across snapshots. For example, `IncrementalAppendScan` could be considered as one of cases that outputs scan tasks across snapshots, which doesn't need the snapshot id.
   
   One solution is to add another layer of interface to support the net changes across snapshots. For example, we can use `ChangelogScanTask` as a parent for both cases of one snapshot and multiple snapshots, and then create a new interface `SnapshotChangelogScanTask` to extend it for use case of a single snapshot. 
   
   I think the current solution is good enough for CDC implementation. But, since it is an interface change, we may want to make sure it is extendable for the future use cases. If it is something we will do to support net changes across snapshots, we may choose the right the right hierarchy and names now.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893929732


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   If we output pos and data file names in the result, we can understand the rows being deleted multiple times, and deduplicate them while merging. In that way, we don't have to change anything in the scan task. Oh, that doesn't work for the case that we don't merge snapshots which deletes the same row multiple times. Never mind.



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910412896


##########
api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by removing a data file from the table.
+ * <p>
+ * Note that all historical delete files added before must be applied while reading the data file to
+ * output only those data records that were live when the data file was removed.
+ */
+public interface DeletedDataFileScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of {@link DeleteFile delete files} to apply when reading the task's data file.
+   *
+   * @return a list of delete files to apply
+   */
+  List<DeleteFile> deletes();

Review Comment:
   I feel like this is related to our "existingDeletes" in the DeletedRowsScan task than it is to `deletes` in `AddedRowScanTask`. Maybe worth changing the name?
   
   Just trying to differentiate between files marking rows which have been deleted so we aren't rendering them and files marking rows which were deleted in the past so we aren't rendering them.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909084369


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {

Review Comment:
   Let me make sure I understand correctly...
   
   Say I have a table/partition with files `A`, `B`, and `C`. Then in `S1` I add a position delete file, `X`, that removes from `B`, and finally add an equality delete file, `Y` in `S2`. For changes from `S1` to `S2` (inclusive), this will produce:
   
   * `DeletedRowsScanTask(file=B, added-deletes=[X], existing-deletes=[], snapshot=S1)`
   * `DeletedRowsScanTask(file=A, added-deletes=[Y], existing-deletes=[], snapshot=S2)`
   * `DeletedRowsScanTask(file=B, added-deletes=[Y], existing-deletes=[X], snapshot=S2)`
   * `DeletedRowsScanTask(file=C, added-deletes=[Y], existing-deletes=[], snapshot=S2)`
   
   Is that correct? An example in Javadoc would be helpful.



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909082179


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends ScanTask {
+  /**
+   * Returns the operation type of records produced by this task (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative change order in which the changes must be applied.
+   */
+  int changeOrder();

Review Comment:
   What about `changeOrdinal`, since this is an ordinal number?
   
   Also, what is the scope of this ordinal? Is it just within the rows produced by this scan?



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


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r910416942


##########
api/src/main/java/org/apache/iceberg/IncrementalScan.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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;
+
+/**
+ * API for configuring an incremental scan.
+ */
+public interface IncrementalScan<ThisT, T extends ScanTask, G extends ScanTaskGroup<T>> extends Scan<ThisT, T, G> {
+  /**
+   * Refines this scan with a start snapshot ID (inclusive).

Review Comment:
   We use "start" and "end" in these java docs but the methods are all called "from" and "to". We should probably align these or at least make it clear.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623640


##########
api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by removing a data file from the table.
+ * <p>
+ * Note that all historical delete files added before must be applied while reading the data file to
+ * output only those data records that were live when the data file was removed.
+ */
+public interface DeletedDataFileScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of {@link DeleteFile delete files} to apply when reading the task's data file.
+   *
+   * @return a list of delete files to apply
+   */
+  List<DeleteFile> deletes();

Review Comment:
   Changed.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623287


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   An exception sounds reasonable. Using nulls in a public API is also not the best idea.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895346775


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   It has nothing to do with primary key(s). Iceberg allows two or multiple rows to be identical. For example,  here are two rows with the same values in each columns(id, name, age).
   ```
   r1 (1, "Alice", 20) at pos 0 of data file df1
   r2 (1, "Alice", 20) at pos 1 of data file df1
   ```
   Assume both s1 and s2 delete r1,  then we will emit two delete records.
   ```
   (1, "Alice", 20, "D", "S1")
   (1, "Alice", 20, "D", "S2")
   ```
   The user won't be able to distinguish the following two scenarios. 
   1. Both deleted r1.
   2. One deleted r1, while another deleted r2.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r900530219


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   After thinking more about this, I agree that commit snapshot ID doesn't make sense for net changes across multiple snapshots in some cases. For instance, when we have 2 snapshots that add delete files for the same data file. In this case, we will have a single task with one data file and two deletes added in multiple snapshots.
   
   Having said that, I am still not sure having separate tasks is a good idea. Apart from more tasks to support in readers, what schema to produce? Our `changelog` metadata table should have the same schema independently whether we include only net changes or all changes per each snapshot.
   
   We can make the commit snapshot ID null when only net changes are produced, skip the commit snapshot ID in both cases or offer some sort of a map of metadata where the snapshot ID may or may not be set.
   
   Let me think more about this.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881947028


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);

Review Comment:
   I assume it returns data files with pos delete and/or eq delete files. I think we need another method that returns the data files with pos deletes file only, since it is an option to handle eq delete separately.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887273561


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public enum ChangelogOperation {

Review Comment:
   Can somebody confirm that `changelog` is one word?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r887377497


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   @flyrain @stevenzwu, there was a question whether we should have common hierarchy for this scan and the incremental append scan. I am not sure it is a good idea as those two are fairly different. They are both incremental, though. We could consider having `IncrementalScan` but then `IncrementalAppendScan` is going to be empty.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889128056


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   > We could consider having IncrementalScan but then IncrementalAppendScan is going to be empty
   
   That was our original plan too. @rdblue was preferring to that we do the refactoring when we actually come to the incremental changelog scan interface. so this PR might be a good place to make the change.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889231846


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   Snapshot sequence numbers are present only in v2 tables. Also, some records may have different commit orders within one snapshot. Consider a Flink upsert that adds a data file and a position delete file against it. Added records should have a smaller commit order (if we decide to output such records in the future).



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889240768


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();

Review Comment:
   Resolving as it no longer applies.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888467260


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -0,0 +1,24 @@
+/*
+ * 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;
+
+public enum ChangelogOperation {

Review Comment:
   It's not in any dictionary AFAIK, but there is a wiki page for that, https://en.wikipedia.org/wiki/Changelog. I'm OK with either `ChangeLog` or `Changelog`.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888553059


##########
api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java:
##########
@@ -0,0 +1,69 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a scan for table changes.
+ */
+public interface IncrementalChangelogScan extends Scan<IncrementalChangelogScan> {

Review Comment:
   That's a good point. The interface `IncrementalAppendScan` doesn't have any method specified for `Append`. How about replacing `IncrementalAppendScan` with `IncrementalScan`, so that `IncrementalChangelogScan` can extend `IncrementalScan`? And we can have `BaseIncrementalAppendScan` implements `IncrementalScan` directly. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889230820


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   > Option 1: Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc. Minimal code change, but messy. We will see DeleteDataFile.
   
   I don't think we will have to add `DeleteDataFile`. We can simply construct `DataFile` from `DeleteFile` like we construct `DataFile` from `ManifestFile` in metadata tables today.
   
   ```
   DataFiles.fromManifest(manifest)
   ```
   
   It is a little bit weird, though, I agree, hence the debate.
   
   > Option 2: Refactor FileScanTask hierarchy.
   
   I'd be up for refactoring `FileScanTask` even though it will require quite some changes. Any concerns anyone?



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888238729


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Follow the discussing on https://github.com/apache/iceberg/pull/4812#discussion_r887384420, there's a discussion on whether we can and should refactor FileScanTask.  This is today's situation, baked into the code and used in many places:
   
   1. Scan returns FileScanTask
   2. FileScanTask has methods DataFile file() and List<DeleteFile> deleteFiles(), which makes sense only in case of scanning data files.
   3. BaseFileScanTask has method Iterable<FileScanTask> split(), which uses file() to calculate the splits.
   
   Problem: for growing number of cases , like CDC, read position deletes, writing a new scan implementation forces FileScanTask to be returned, which will then have to unnaturally wrap non-data files as DataFile file().  In these cases, even wrapping a DeleteFile as a DataFile.
   
   I see two options
   
   Option 1:  Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc.  Minimal code change, but messy.  We will see DeleteDataFile.
   
   Option 2:  Refactor FileScanTask hierarchy.
   One potential solution is in #4812:
   
   1. Scan still returns FileScanTask (cant change this)
   2. FileScanTask now becomes a base class with method : ContentFile<T> contentFile()
   3. BaseFileScanTask will do all the splitting logic based on contentFile().  In fact, this is a small code replacement of using file() => contentFile() as already today all the methods it uses are in fact from contentFile and not specific to DataFile.
   4. Eventually, we make FileScanTask subtype DataFileScanTask with methods DataFile dataFile() and List<DeleteFile> deleteFiles(), and code that deals exclusively with data files should use this scan task for clarity. 
   5. Deprecate current FileScanTask file() and deleteFiles() and eventually remove them in favor of the subclass (though with 1.0, we may need to wait for next major relesae to do this).
   
   With this solution, we need only minimal changes for 1-3, and can do more refactoring of 4-5 in different prs gradually.  It also preserves backward compatibility.  I think this solution would be clearer in the code, where we are not dealing exclusively with DataFile but rather different content files.
   
   But I am not entirely sure what kind of interface CDC FileScanTask will need, so will be open to see if this works or what you or others have thought.
   
   Also, let me know if this is the not the place you meant for this discussion. 



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


[GitHub] [iceberg] szehon-ho commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
szehon-ho commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r888238729


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Follow the discussing on https://github.com/apache/iceberg/pull/4812#discussion_r887384420, there's a discussion on whether we can and should refactor FileScanTask.  This is today's situation, baked into the code and used in many places:
   
   1. Scan returns FileScanTask
   2. FileScanTask has methods DataFile file() and List<DeleteFile> deleteFiles(), which makes sense only in case of scanning data files.
   3. BaseFileScanTask has method Iterable<FileScanTask> split(), which uses file() to calculate the splits.
   
   Problem: for growing number of cases , like CDC, read position deletes, writing a new scan implementation forces FileScanTask to be returned, which will then have to unnaturally wrap non-data files as DataFile file().  In these cases, even wrapping a DeleteFile as a DataFile.
   
   I see two options
   
   Option 1:  Leave it as is, implement various subclasses of DataFile that represent delete files, change logs, etc.  Minimal code change, but messy.  We will see DeleteDataFile.
   
   Option 2:  Refactor FileScanTask hierarchy.
   One potential solution is in #4812:
   
   1. Scan still returns FileScanTask (cant change this)
   2. FileScanTask now becomes a parameterized base class with method : ContentFile<T> contentFile()
   3. BaseFileScanTask will do all the splitting logic based on contentFile().  In fact, this is a small code replacement of using file() => contentFile() as already today all the methods it uses are in fact from contentFile and not specific to DataFile.
   4. Eventually, we make DataFileScanTask that has methods DataFile dataFile() and List<DeleteFile> deleteFiles(), and code that deals exclusively with DataFileScanTask can use that for clarity. 
   5. Deprecate current FileScanTask file() and deleteFiles() and eventually remove them in favor of the subclass (though with 1.0, not sure the timeline we can do this).
   
   With this solution, we need only minimal changes for 1-3, and can do more refactoring of 4 in different prs, while preserving backward compatibility.  I like this solution as it keeps some kind of clarity in the code that we are not dealing with DataFile but rather different content files.
   
   But I am not entirely sure what kind of interface CDC FileScanTask will need, so will be open to see if this works.
   
   Also, let me know if this is the not the place you meant for this discussion. 



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r889736363


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   Refactoring `FileScanTask` sounds good to me too. Maybe as a separate PR before this?



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r890312010


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();

Review Comment:
   Change order sounds good to me.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r893929732


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   If we output pos and data file names in the result, we can understand the rows being deleted multiple times, and deduplicate them while merging. In that way, we don't have to change anything in the scan task. Oh, that doesn't solve the issue if we don't merge snapshots which deletes the same row multiple times. Never mind, it won't work for all use cases. 



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r891689671


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   @stevenzwu, not sure yet. I guess that would be the correct behavior but may complicate things quite a bit as we need to load historic deletes too.



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895077955


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   > The problem happens when we only generate CDC records from s2
   
   I feel it is an invalid case. CDC read should been incremental and continuous. If an Iceberg table is the CDC source, I would imagine CDC read should start with full table scan, then transition to the incremental changelog scan. We shouldn't read S2 without reading S1. Even if CDC read started from S2 and continues forward, it is also ok to emit a delete in my mind.
   
   > The edge case is that there are two same rows in the data files(r1, r2). We don't know whether r1 is deleted twice or both r1 and r2 are deleted if we output duplicated delete cdc records
   
   I didn't quite get this edge case. Do r1 and r2 have the same primary key and the data file is removed?
   



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r895077955


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   > The problem happens when we only generate CDC records from s2
   
   I feel it is an invalid case. CDC read should been incremental and continuous. If an Iceberg table is the CDC source, I would imagine CDC read should start with full table scan, then transition to the incremental changelog scan. We shouldn't read S2 without reading S1.
   
   > The edge case is that there are two same rows in the data files(r1, r2). We don't know whether r1 is deleted twice or both r1 and r2 are deleted if we output duplicated delete cdc records
   
   I didn't quite get this edge case. Do r1 and r2 have the same primary key and the data file is removed?
   



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r908667211


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {

Review Comment:
   No longer applies.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881906894


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {

Review Comment:
   I thought about this but I consider `Changelog` as something that already contains the records (e.g. the result of scanning these files).



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881949205


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);
+
+    /**
+     * Returns scan tasks for equality delete files that were not assigned to matching data files in a given snapshot.
+     */
+    Iterable<FileScanTask> unresolvedEqualityDeleteFiles(long snapshotId);

Review Comment:
   The idea is that this method would only return equality deletes not assigned to data files (if `resolveEqualityDeletes` is not called). If we assign equality deletes, then this will be empty. That's why the special name.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881949205


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);
+
+    /**
+     * Returns scan tasks for equality delete files that were not assigned to matching data files in a given snapshot.
+     */
+    Iterable<FileScanTask> unresolvedEqualityDeleteFiles(long snapshotId);

Review Comment:
   This method returns only equality deletes not assigned to data files (if `resolveEqualityDeletes` is not called). If we assign equality deletes, then this will be empty. That's why the special name.



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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881945543


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files removed in a given snapshot.
+     */
+    Iterable<FileScanTask> removedDataFiles(long snapshotId);
+
+    /**
+     * Returns scan tasks for data files that have at least one matching delete file added in a given snapshot.
+     */
+    Iterable<FileScanTask> dataFilesWithDeletes(long snapshotId);
+
+    /**
+     * Returns scan tasks for equality delete files that were not assigned to matching data files in a given snapshot.
+     */
+    Iterable<FileScanTask> unresolvedEqualityDeleteFiles(long snapshotId);

Review Comment:
   IIUC, these are purely eq delete files. We may just call it `equalityDeleteFiles`?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r882092591


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();
+
+  /**
+   * An index for looking up file scan tasks for changes in particular snapshots.
+   */
+  interface FileChangeIndex {
+    /**
+     * Returns a list of snapshots that are covered by this index.
+     */
+    List<Snapshot> snapshots();
+
+    /**
+     * Returns scan tasks for data files added in a given snapshot.
+     */
+    Iterable<FileScanTask> addedDataFiles(long snapshotId);

Review Comment:
   should the return type be `CloseableIterable`?



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


[GitHub] [iceberg] stevenzwu commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
stevenzwu commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r882088977


##########
api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java:
##########
@@ -20,6 +20,8 @@
 
 package org.apache.iceberg;
 
+import org.apache.iceberg.io.CloseableIterable;
+
 /**
  * API for configuring an incremental table scan for appends only snapshots

Review Comment:
   the hierarchy should be the other way around. `ChangeScan` extends `IncrementalAppendScan` as change scan is the more complex superset.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r881925598


##########
api/src/main/java/org/apache/iceberg/ChangeScan.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * 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;
+
+/**
+ * API for configuring a table scan for changes.
+ */
+public interface ChangeScan extends Scan<TableScan> {
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (inclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotInclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes starting from a particular snapshot (exclusive).
+   * <p>
+   * If the start snapshot is not configured, it is defaulted to the oldest ancestor
+   * of the end snapshot (inclusive).
+   *
+   * @param fromSnapshotId the start snapshot ID (exclusive)
+   * @return this for method chaining
+   */
+  ChangeScan fromSnapshotExclusive(long fromSnapshotId);
+
+  /**
+   * Instructs this scan to look for changes up to a particular snapshot (inclusive).
+   * <p>
+   * If the end snapshot is not configured, it is defaulted to the current table snapshot (inclusive).
+   *
+   * @param toSnapshotId the end snapshot ID (inclusive)
+   * @return this for method chaining
+   */
+  ChangeScan toSnapshot(long toSnapshotId);
+
+  /**
+   * Instructs this scan to assign equality delete files to matching data files,
+   * which may be expensive if equality deletes are not scoped to particular partitions.
+   *
+   * @return this for method chaining
+   */
+  ChangeScan resolveEqualityDeletes();
+
+  /**
+   * Plans scan tasks for all file changes.
+   *
+   * @return this for method chaining
+   */
+  FileChangeIndex planFiles();

Review Comment:
   One alternative would be to add new `FileScanTask`s for different types but that's going to be ugly to navigate and consume, in my view.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r900535298


##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,30 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask {

Review Comment:
   Without applying historical deletes, the changelog may tell the user that a particular snapshot deleted a record that was wasn't live and was removed in an earlier operation. That can happen both with equality and position deletes.



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


[GitHub] [iceberg] aokolnychyi commented on pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#issuecomment-1159115451

   We discussed refactoring `FileScanTask` using a parent interface during the sync. I gave it a try in #5077.
   I will need a little bit more time to iterate on it but it is open for early feedback.


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


[GitHub] [iceberg] flyrain commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909111695


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   > I think the point is that we can read data files normally and apply deletes to easily produce the squashed version, which could be quite valuable to end users.
   
   Yes. That's one of the use case we think of, which could be resolved by separating the tasks. 
   
   Here is another one, which is a bit different from the above use case, `S1` adds rows with ids 4, 5, and 6, then `S2` deletes the rows with id 5 and 7, `S2` also adds the rows with id 8, 9. We want to have a single `AddedRowsScanTask` to get all added rows with s1 and s2. To give one snapshot id doesn't make sense since added rows 4, 6, 8, 9 come from different snapshots. Of course, we can archive the same result by splitting it into two `AddedRowsScanTask`s, then merge them. But a task across multiple multiple snapshots may have perf advantage, or simpler implementation for that use case. What do you think?



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


[GitHub] [iceberg] rdblue commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r909810635


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   @flyrain, for that case, I think we need to solve the problem differently. The new rows from S2 are in a separate file, so we should create a specific task for that file with the correct metadata. Then we can use a task group to combine the tasks into one for performance. That gives us the correct metadata and is nearly identical in terms of performance.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623287


##########
api/src/main/java/org/apache/iceberg/ChangelogScanTask.java:
##########
@@ -0,0 +1,40 @@
+/*
+ * 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;
+
+/**
+ * A changelog scan task.
+ */
+public interface ChangelogScanTask extends FileScanTask {
+  /**
+   * Returns the operation type (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the relative commit order in which the changes must be applied.
+   */
+  int commitOrder();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();

Review Comment:
   An exception sounds reasonable. Using a null in a public API is probably not the best idea.



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


[GitHub] [iceberg] aokolnychyi commented on a diff in pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
aokolnychyi commented on code in PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#discussion_r911623849


##########
api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java:
##########
@@ -0,0 +1,42 @@
+/*
+ * 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;
+
+/**
+ * A scan task for inserted data records generated by adding a data file to the table.
+ * <p>
+ * Note that added files may have matching delete files. If so, such delete files will be included

Review Comment:
   Explained.



##########
api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java:
##########
@@ -0,0 +1,49 @@
+/*
+ * 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;
+
+/**
+ * A scan task for deleted data records generated by adding delete files to the table.

Review Comment:
   Added.



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


[GitHub] [iceberg] rdblue commented on pull request #4870: API: Add a scan for changes

Posted by GitBox <gi...@apache.org>.
rdblue commented on PR #4870:
URL: https://github.com/apache/iceberg/pull/4870#issuecomment-1173162910

   Looks great. Thanks for all the work and discussion, everyone!


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