You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2022/07/03 20:05:00 UTC

[iceberg] branch master updated: API: Add a scan for changes (#4870)

This is an automated email from the ASF dual-hosted git repository.

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new c69a3dd61 API: Add a scan for changes (#4870)
c69a3dd61 is described below

commit c69a3dd6171ea2e6d2930be3a0402335d696354b
Author: Anton Okolnychyi <ao...@apple.com>
AuthorDate: Sun Jul 3 13:04:55 2022 -0700

    API: Add a scan for changes (#4870)
---
 .../java/org/apache/iceberg/AddedRowsScanTask.java | 53 ++++++++++++++++++
 .../org/apache/iceberg/ChangelogOperation.java     | 27 ++++++++++
 .../java/org/apache/iceberg/ChangelogScanTask.java | 41 ++++++++++++++
 .../apache/iceberg/DeletedDataFileScanTask.java    | 52 ++++++++++++++++++
 .../org/apache/iceberg/DeletedRowsScanTask.java    | 62 ++++++++++++++++++++++
 .../org/apache/iceberg/IncrementalAppendScan.java  | 38 +------------
 .../apache/iceberg/IncrementalChangelogScan.java   | 27 ++++++++++
 .../java/org/apache/iceberg/IncrementalScan.java   | 59 ++++++++++++++++++++
 api/src/main/java/org/apache/iceberg/Table.java    | 11 ++++
 9 files changed, 333 insertions(+), 37 deletions(-)

diff --git a/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java
new file mode 100644
index 000000000..ba7f73047
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/AddedRowsScanTask.java
@@ -0,0 +1,53 @@
+/*
+ * 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 inserts generated by adding a data file to the table.
+ * <p>
+ * Note that added data files may have matching delete files. This may happen if a matching position
+ * delete file is committed in the same snapshot or if changes for multiple snapshots are squashed together.
+ * <p>
+ * Suppose snapshot S1 adds data files F1, F2, F3 and a position delete file, D1, that marks particular
+ * records in F1 as deleted. A scan for changes generated by S1 should include the following tasks:
+ * <ul>
+ *   <li>AddedRowsScanTask(file=F1, deletes=[D1], snapshot=S1)</li>
+ *   <li>AddedRowsScanTask(file=F2, deletes=[], snapshot=S1)</li>
+ *   <li>AddedRowsScanTask(file=F3, deletes=[], snapshot=S1)</li>
+ * </ul>
+ * <p>
+ * Readers consuming these tasks should produce added records with metadata like change ordinal and
+ * commit snapshot ID.
+ */
+public interface AddedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of {@link DeleteFile delete files} to apply when reading the data file in this task.
+   *
+   * @return a list of delete files to apply
+   */
+  List<DeleteFile> deletes();
+
+  @Override
+  default ChangelogOperation operation() {
+    return ChangelogOperation.INSERT;
+  }
+}
diff --git a/api/src/main/java/org/apache/iceberg/ChangelogOperation.java b/api/src/main/java/org/apache/iceberg/ChangelogOperation.java
new file mode 100644
index 000000000..3d6ad72c9
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/ChangelogOperation.java
@@ -0,0 +1,27 @@
+/*
+ * 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;
+
+/**
+ * An enum representing possible operations in a changelog.
+ */
+public enum ChangelogOperation {
+  INSERT, DELETE
+}
diff --git a/api/src/main/java/org/apache/iceberg/ChangelogScanTask.java b/api/src/main/java/org/apache/iceberg/ChangelogScanTask.java
new file mode 100644
index 000000000..f74fea747
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/ChangelogScanTask.java
@@ -0,0 +1,41 @@
+/*
+ * 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 type of changes produced by this task (i.e. insert/delete).
+   */
+  ChangelogOperation operation();
+
+  /**
+   * Returns the ordinal of changes produced by this task. This number indicates the order in which
+   * changes produced by this scan must be applied. Operations with a lower ordinal must be applied first.
+   */
+  int changeOrdinal();
+
+  /**
+   * Returns the snapshot ID in which the changes were committed.
+   */
+  long commitSnapshotId();
+}
diff --git a/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java
new file mode 100644
index 000000000..0ef5f0946
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/DeletedDataFileScanTask.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg;
+
+import java.util.List;
+
+/**
+ * A scan task for deletes generated by removing a data file from the table.
+ * <p>
+ * Note that all historical delete files added earlier must be applied while reading the data file.
+ * This is required to output only those data records that were live when the data file was removed.
+ * <p>
+ * Suppose snapshot S1 contains data files F1, F2, F3. Then snapshot S2 adds a position delete file, D1,
+ * that deletes records from F2 and snapshot S3 removes F2 entirely. A scan for changes generated by S3
+ * should include the following task:
+ * <ul>
+ *   <li>DeletedDataFileScanTask(file=F2, existing-deletes=[D1], snapshot=S3)</li>
+ * </ul>
+ * <p>
+ * Readers consuming these tasks should produce deleted records with metadata like change ordinal and
+ * commit snapshot ID.
+ */
+public interface DeletedDataFileScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of previously added {@link DeleteFile delete files} to apply when reading the data file in this task.
+   *
+   * @return a list of delete files to apply
+   */
+  List<DeleteFile> existingDeletes();
+
+  @Override
+  default ChangelogOperation operation() {
+    return ChangelogOperation.DELETE;
+  }
+}
diff --git a/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java
new file mode 100644
index 000000000..cd8ebfac8
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/DeletedRowsScanTask.java
@@ -0,0 +1,62 @@
+/*
+ * 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 deletes generated by adding delete files to the table.
+ * <p>
+ * Suppose snapshot S1 contains data files F1, F2, F3. Then snapshot S2 adds a position delete file, D1,
+ * that deletes records from F2 and snapshot S3 adds an equality delete file, D2, that removes records
+ * from F1, F2, F3. A scan for changes from S2 to S3 (inclusive) should include the following tasks:
+ * <ul>
+ *   <li>DeletedRowsScanTask(file=F2, added-deletes=[D1], existing-deletes=[], snapshot=S2)</li>
+ *   <li>DeletedRowsScanTask(file=F1, added-deletes=[D2], existing-deletes=[], snapshot=S3)</li>
+ *   <li>DeletedRowsScanTask(file=F2, added-deletes=[D2], existing-deletes=[D1], snapshot=S3)</li>
+ *   <li>DeletedRowsScanTask(file=F3, added-deletes=[D2], existing-deletes=[], snapshot=S3)</li>
+ * </ul>
+ * <p>
+ * Readers consuming these tasks should produce deleted records with metadata like change ordinal and
+ * commit snapshot ID.
+ */
+public interface DeletedRowsScanTask extends ChangelogScanTask, ContentScanTask<DataFile> {
+  /**
+   * A list of added {@link DeleteFile delete files} that apply to the task's data file.
+   * Records removed by these delete files should appear as deletes in the changelog.
+   *
+   * @return a list of added delete files
+   */
+  List<DeleteFile> addedDeletes();
+
+  /**
+   * A list of {@link DeleteFile delete files} that existed before and must be applied prior to
+   * determining which records are deleted by delete files in {@link #addedDeletes()}.
+   * Records removed by these delete files should not appear in the changelog.
+   *
+   * @return a list of existing delete files
+   */
+  List<DeleteFile> existingDeletes();
+
+  @Override
+  default ChangelogOperation operation() {
+    return ChangelogOperation.DELETE;
+  }
+}
diff --git a/api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java b/api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java
index 2817a16b4..24b9ed7cc 100644
--- a/api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java
+++ b/api/src/main/java/org/apache/iceberg/IncrementalAppendScan.java
@@ -23,41 +23,5 @@ package org.apache.iceberg;
 /**
  * API for configuring an incremental table scan for appends only snapshots
  */
-public interface IncrementalAppendScan extends Scan<IncrementalAppendScan, FileScanTask, CombinedScanTask> {
-
-  /**
-   * Refine the incremental scan with the start snapshot inclusive.
-   * <p>
-   * If neither {@link #fromSnapshotInclusive(long)} or {@link #fromSnapshotExclusive(long)} is provided,
-   * start snapshot inclusive is defaulted to the oldest ancestor of the end snapshot.
-   *
-   * @param fromSnapshotId the start snapshot id inclusive
-   * @return an incremental table scan from {@code fromSnapshotId} inclusive
-   * @throws IllegalArgumentException if the start snapshot is not an ancestor
-   * of the end snapshot
-   */
-  IncrementalAppendScan fromSnapshotInclusive(long fromSnapshotId);
-
-  /**
-   * Refine the incremental scan with the start snapshot exclusive.
-   * <p>
-   * If neither {@link #fromSnapshotInclusive(long)} or {@link #fromSnapshotExclusive(long)} is provided,
-   * start snapshot inclusive is defaulted to the oldest ancestor of the end snapshot.
-   *
-   * @param fromSnapshotId the start snapshot id (exclusive)
-   * @return an incremental table scan from {@code fromSnapshotId} exclusive
-   * @throws IllegalArgumentException if the start snapshot is not an ancestor
-   * of the end snapshot
-   */
-  IncrementalAppendScan fromSnapshotExclusive(long fromSnapshotId);
-
-  /**
-   * Refine the incremental scan with the end snapshot inclusive.
-   * <p>
-   * If not provided, end snapshot is defaulted to the current table snapshot.
-   *
-   * @param toSnapshotId the end snapshot id (inclusive)
-   * @return an incremental table scan up to {@code toSnapshotId} inclusive
-   */
-  IncrementalAppendScan toSnapshot(long toSnapshotId);
+public interface IncrementalAppendScan extends IncrementalScan<IncrementalAppendScan, FileScanTask, CombinedScanTask> {
 }
diff --git a/api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java b/api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java
new file mode 100644
index 000000000..169695058
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/IncrementalChangelogScan.java
@@ -0,0 +1,27 @@
+/*
+ * 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 IncrementalScan<IncrementalChangelogScan, ChangelogScanTask, ScanTaskGroup<ChangelogScanTask>> {
+}
diff --git a/api/src/main/java/org/apache/iceberg/IncrementalScan.java b/api/src/main/java/org/apache/iceberg/IncrementalScan.java
new file mode 100644
index 000000000..501c11bc1
--- /dev/null
+++ b/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> {
+  /**
+   * 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
+   * @throws IllegalArgumentException if the start snapshot is not an ancestor of the end snapshot
+   */
+  ThisT 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
+   * @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).
+   * <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
+   */
+  ThisT toSnapshot(long toSnapshotId);
+}
diff --git a/api/src/main/java/org/apache/iceberg/Table.java b/api/src/main/java/org/apache/iceberg/Table.java
index 1e9667f4c..b83f07c6f 100644
--- a/api/src/main/java/org/apache/iceberg/Table.java
+++ b/api/src/main/java/org/apache/iceberg/Table.java
@@ -64,6 +64,17 @@ public interface Table {
     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() {
+    throw new UnsupportedOperationException("Incremental changelog scan is not supported");
+  }
+
   /**
    * Return the {@link Schema schema} for this table.
    *