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/10/18 20:02:13 UTC

[GitHub] [iceberg] flyrain opened a new pull request, #6012: Add a procedure to generate table changes

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

   Add a procedure to generate table changes. Here are changes in this PR.
   1. Defines the user interface.
   2. Generates update pre-image and post-image when user provide the identifier columns.
   3. Uses the window function instead of joining for better performance.
   
   cc @aokolnychyi @rdblue @szehon-ho @kbendick @anuragmantri @karuppayya @chenjunjiedada 


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1117781968


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {

Review Comment:
   Tested with `dropDuplicates(columns)`.  It doesn't fit here due to following reasons
   1. It keeps one duplications. for example, we want to remove both rows, (1, 'data', 'DETELE') (1, 'data', 'INSERT'), but it keeps one.
   2. In case of duplicated rows in the iceberg table, we still want to keep them. for example, we got 4 rows: (1, 'data', 'DETELE') (1, 'data', 'INSERT'), (1, 'data', 'INSERT'), (1, 'data', 'INSERT'). We want to only remove the first two rows. The `dropDuplicates` will remove all except keeping one. This is the behavior we don't want.
   
   The iterator meet our requirements here. I will explore the window function. It should be faster.



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1444629617

   Thanks a lot for the detailed review, @aokolnychyi ! Resolved them all and ready for another look.


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1122662484


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {

Review Comment:
   You are right, let's consider the best way for removing carryovers separately. Out of scope for 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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      if (hasIdentifierColumns(args)) {
+        String[] identifierColumns = args.getString(5).split(",");
+
+        if (identifierColumns.length > 0) {
+          Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+          df = transform(df, repartitionColumns);
+        }
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);

Review Comment:
   Here we only remove carryover rows, without computing the updated rows. The iterator is built for both, it will check if they are updated rows and check if they are carryover rows. The first check is not necessary. We only need the second one to see if two rows are identical. If yes, they are carryover rows, we remove 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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),

Review Comment:
   Make sense to allow `Dataframe` to consume timestamp range. Will create a followup PR for that. For this procedure, we still need all these parameter, right? What do you mean by replacing all of them with generic `options`?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {

Review Comment:
   Made the change per suggestion.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      if (hasIdentifierColumns(args)) {
+        String[] identifierColumns = args.getString(5).split(",");
+
+        if (identifierColumns.length > 0) {
+          Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+          df = transform(df, repartitionColumns);
+        }
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);

Review Comment:
   i'm not sure I understand the comment here, don't we have the iterator so we don't need to do 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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112274302


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   Looking a bit more. Not only `SparkSessionCatalog` works, `SparkCatalog` works as well.  My test `TestGenerateChangesProcedure` inherits from `SparkCatalogTestBase`, which covers 3 types of catalog. They all work well. Am I missing something here?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112414211


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {

Review Comment:
   The branch has been removed. Let me know if there is anything I missed.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1116279838


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns

Review Comment:
   nit: What about `pre/post update images` instead of `update-row` in all places in this doc?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {

Review Comment:
   Can we add some Javadoc on what this procedure does?
   May make sense to add a few details about the algorithm too.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {

Review Comment:
   nit: The definition order in other procedures: static vars, static methods, constructor, instance methods. Could you move this method below the static constants? 



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {

Review Comment:
   I'd consider supporting these parameters (names TBD).
   - Table name
   - Changelog view name
   - A map of options that would also contain boundaries
   - A boolean to indicate whether we should compute pre/post images
   - A boolean to indicate whether we should remove carried over records
   - Default identifier columns to use if not defined in snapshot's schema



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Each snapshot's schema has a list of identity columns. I feel asking users for identity columns when those are already defined is not necessary. Let me think how we can handle this without too much complication.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   I don't think operation conditions should matter.
   
   Suppose I have the following table with a primary key column `pk`.
   
   ```
   -----------------
   pk | dep | salary
   -----------------
   1, hr, 100
   2, software, 80
   3, hardware, 120
   4, software, 110
   5, hr, 95
   6, software, 100
   ```
   
   If I issue `UPDATE t SET salary = 110 WHERE salary = 100`, it will generate the following changes.
   
   ```
   --------------------------------------------------------------------------
   _change_type | _change_ordinal | _commit_snapshot_id | pk | dep | salary
   --------------------------------------------------------------------------
   DELETE, 0, s1, 1, hr, 100
   INSERT, 0, s1, 1, hr, 110
   DELETE, 0, s1, 6, software, 100
   INSERT, 0, s1, 6, software, 110
   ```
   
   Even though I had a condition on `salary`, we should use `pk` for computing pre/post images.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java:
##########
@@ -53,6 +53,7 @@ private static Map<String, Supplier<ProcedureBuilder>> initProcedureBuilders() {
     mapBuilder.put("ancestors_of", AncestorsOfProcedure::builder);
     mapBuilder.put("register_table", RegisterTableProcedure::builder);
     mapBuilder.put("publish_changes", PublishChangesProcedure::builder);
+    mapBuilder.put("generate_changes", GenerateChangesProcedure::builder);

Review Comment:
   Are there any alternative names? I am not sure the procedure actually generates changes.
   Let's think a bit. It is not bad but I wonder whether we can be a bit more specific.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    Long[] snapshotIds = getSnapshotIds(tableName, args);
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    DataFrameReader reader = spark().read();
+    if (snapshotIds[0] != null) {
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, snapshotIds[0]);
+    }
+
+    if (snapshotIds[1] != null) {
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, snapshotIds[1]);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private Long[] getSnapshotIds(String tableName, InternalRow args) {
+    Long[] snapshotIds = new Long[] {null, null};
+
+    Long startTimestamp = args.isNullAt(5) ? null : DateTimeUtil.microsToMillis(args.getLong(5));
+    Long endTimestamp = args.isNullAt(6) ? null : DateTimeUtil.microsToMillis(args.getLong(6));
+
+    if (startTimestamp == null && endTimestamp == null) {
+      snapshotIds[0] = args.isNullAt(1) ? null : args.getLong(1);
+      snapshotIds[1] = args.isNullAt(2) ? null : args.getLong(2);
+    } else {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      Snapshot[] snapshots = snapshotsFromTimestamp(startTimestamp, endTimestamp, table);
+
+      if (snapshots != null) {
+        snapshotIds[0] = snapshots[0].parentId();
+        snapshotIds[1] = snapshots[1].snapshotId();
+      }
+    }
+    return snapshotIds;
+  }
+
+  private Snapshot[] snapshotsFromTimestamp(Long startTimestamp, Long endTimestamp, Table table) {
+    Snapshot[] snapshots = new Snapshot[] {null, null};
+
+    if (startTimestamp != null && endTimestamp != null && startTimestamp > endTimestamp) {
+      throw new IllegalArgumentException(
+          "Start timestamp must be less than or equal to end timestamp");
+    }
+
+    if (startTimestamp == null) {
+      snapshots[0] = SnapshotUtil.oldestAncestor(table);
+    } else {
+      snapshots[0] = SnapshotUtil.oldestAncestorAfter(table, startTimestamp);
+    }
+
+    if (endTimestamp == null) {
+      snapshots[1] = table.currentSnapshot();
+    } else {
+      snapshots[1] = table.snapshot(SnapshotUtil.snapshotIdAsOfTime(table, endTimestamp));
+    }
+
+    if (snapshots[0] == null || snapshots[1] == null) {
+      return null;
+    }
+
+    return snapshots;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+    Column[] sortSpec = sortSpec(df, partitionSpec);
+
+    int changeTypeIdx = df.schema().fieldIndex(MetadataColumns.CHANGE_TYPE.name());
+    List<Integer> partitionIdx =
+        Arrays.stream(partitionSpec)
+            .map(column -> df.schema().fieldIndex(column.toString()))
+            .collect(Collectors.toList());
+
+    return df.repartition(partitionSpec)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            processRowsWithinTask(changeTypeIdx, partitionIdx), RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getPartitionSpec(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        partitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            String.format("Identifier column '%s' does not exist in the table", identifiers[i]), e);
+      }
+    }
+    partitionSpec[partitionSpec.length - 1] = df.col(MetadataColumns.CHANGE_ORDINAL.name());
+    return partitionSpec;
+  }
+
+  @NotNull
+  private static Column[] sortSpec(Dataset<Row> df, Column[] partitionSpec) {
+    Column[] sortSpec = new Column[partitionSpec.length + 1];
+    System.arraycopy(partitionSpec, 0, sortSpec, 0, partitionSpec.length);
+    sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name());
+    return sortSpec;
+  }
+
+  private static MapPartitionsFunction<Row, Row> processRowsWithinTask(

Review Comment:
   I'll take a look at this with fresh eyes tomorrow. It would be nice to decouple this into a class so that we can unit test by explicitly passing an iterator of rows.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),

Review Comment:
   I am not sure. I'd consider having `read_options` or `options` as a map that would be passed while loading deletes and inserts as `DataFrame`. Then users can specify boundaries directly in the map.
   
   We already respect these options from `SparkReadOptions` in the `changes` table:
   
   ```
   // Start snapshot ID used in incremental scans (exclusive)
   public static final String START_SNAPSHOT_ID = "start-snapshot-id";
   
   // End snapshot ID used in incremental scans (inclusive)
   public static final String END_SNAPSHOT_ID = "end-snapshot-id";
   ```
   
   We could add `start-timestamp` and `end-timestamp`, `start-snapshot-id-inclusive`.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import static org.apache.iceberg.ChangelogOperation.DELETE;
+import static org.apache.iceberg.ChangelogOperation.INSERT;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    Long[] snapshotIds = getSnapshotIds(tableName, args);
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    DataFrameReader reader = spark().read();
+    if (snapshotIds[0] != null) {
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, snapshotIds[0]);
+    }
+
+    if (snapshotIds[1] != null) {
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, snapshotIds[1]);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private Long[] getSnapshotIds(String tableName, InternalRow args) {
+    Long[] snapshotIds = new Long[] {null, null};
+
+    Long startTimestamp = args.isNullAt(5) ? null : DateTimeUtil.microsToMillis(args.getLong(5));
+    Long endTimestamp = args.isNullAt(6) ? null : DateTimeUtil.microsToMillis(args.getLong(6));
+
+    if (startTimestamp == null && endTimestamp == null) {
+      snapshotIds[0] = args.isNullAt(1) ? null : args.getLong(1);
+      snapshotIds[1] = args.isNullAt(2) ? null : args.getLong(2);
+    } else {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      Snapshot[] snapshots = snapshotsFromTimestamp(startTimestamp, endTimestamp, table);
+
+      if (snapshots != null) {
+        snapshotIds[0] = snapshots[0].parentId();
+        snapshotIds[1] = snapshots[1].snapshotId();
+      }
+    }
+    return snapshotIds;
+  }
+
+  private Snapshot[] snapshotsFromTimestamp(Long startTimestamp, Long endTimestamp, Table table) {
+    Snapshot[] snapshots = new Snapshot[] {null, null};
+
+    if (startTimestamp != null && endTimestamp != null && startTimestamp > endTimestamp) {
+      throw new IllegalArgumentException(
+          "Start timestamp must be less than or equal to end timestamp");
+    }
+
+    if (startTimestamp == null) {
+      snapshots[0] = SnapshotUtil.oldestAncestor(table);
+    } else {
+      snapshots[0] = SnapshotUtil.oldestAncestorAfter(table, startTimestamp);
+    }
+
+    if (endTimestamp == null) {
+      snapshots[1] = table.currentSnapshot();
+    } else {
+      snapshots[1] = table.snapshot(SnapshotUtil.snapshotIdAsOfTime(table, endTimestamp));
+    }
+
+    if (snapshots[0] == null || snapshots[1] == null) {
+      return null;
+    }
+
+    return snapshots;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+    Column[] sortSpec = sortSpec(df, partitionSpec);
+
+    int changeTypeIdx = df.schema().fieldIndex(MetadataColumns.CHANGE_TYPE.name());
+    List<Integer> partitionIdx =
+        Arrays.stream(partitionSpec)
+            .map(column -> df.schema().fieldIndex(column.toString()))
+            .collect(Collectors.toList());
+
+    return df.repartition(partitionSpec)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            processRowsWithinTask(changeTypeIdx, partitionIdx), RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getPartitionSpec(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        partitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            String.format("Identifier column '%s' does not exist in the table", identifiers[i]), e);
+      }
+    }
+    partitionSpec[partitionSpec.length - 1] = df.col(MetadataColumns.CHANGE_ORDINAL.name());
+    return partitionSpec;
+  }
+
+  @NotNull
+  private static Column[] sortSpec(Dataset<Row> df, Column[] partitionSpec) {
+    Column[] sortSpec = new Column[partitionSpec.length + 1];
+    System.arraycopy(partitionSpec, 0, sortSpec, 0, partitionSpec.length);
+    sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name());
+    return sortSpec;
+  }
+
+  private static MapPartitionsFunction<Row, Row> processRowsWithinTask(
+      int changeTypeIndex, List<Integer> partitionIdx) {
+    return rowIterator -> {
+      Iterator<Row> iterator =
+          new Iterator<Row>() {
+            private Row nextRow = null;
+
+            @Override
+            public boolean hasNext() {
+              if (nextRow != null) {
+                return true;
+              }
+              return rowIterator.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              // if the next row is cached and changed, return it directly
+              if (nextRow != null
+                  && !nextRow.getString(changeTypeIndex).equals(DELETE.name())
+                  && !nextRow.getString(changeTypeIndex).equals(INSERT.name())) {
+                Row row = nextRow;
+                nextRow = null;
+                return row;
+              }
+
+              Row currentRow = currentRow();
+
+              if (rowIterator.hasNext()) {
+                GenericRowWithSchema nextRow = (GenericRowWithSchema) rowIterator.next();
+
+                if (withinPartition(currentRow, nextRow)
+                    && currentRow.getString(changeTypeIndex).equals(DELETE.name())
+                    && nextRow.getString(changeTypeIndex).equals(INSERT.name())) {

Review Comment:
   I think creating a separate class for this functionality will allow us to unit test it.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1084590470


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   Defaulting the name will only work in the session catalog but I am not sure we have to do anything about it. Other catalogs will not support views.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   What about `changelog_view`, `changelog_view_name` or similar?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())

Review Comment:
   This name should probably match the input arg name.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {

Review Comment:
   I am not sure about the name. I don't have a great alternative but it does not seem to me like we generate changes in this procedure. It seems more like we register a changelog view. Any alternatives?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {

Review Comment:
   nit: To me, it would be better to pull the needed arguments before calling this method rather than passing the row here and doing the extraction within the method itself.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");

Review Comment:
   I don't think we should proceed with the execution if the user asked to compute pre/post images but that's not possible.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {

Review Comment:
   What if the provided identifier columns don't match with identifier columns defined on one of the scan snapshots?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    // no need to validate the read options here since the reader will validate them
+    DataFrameReader reader = spark().read();
+    reader.options(readOptions(args));
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                if (k.toString().equals(SparkReadOptions.START_TIMESTAMP)
+                    || k.toString().equals(SparkReadOptions.END_TIMESTAMP)) {
+                  options.put(k.toString(), toMillis(v.toString()));

Review Comment:
   We shouldn't do any conversion here. If we want to support timestamps, we should add support to the reader. It is possible by adding new functionality to `SparkReadConf`. For now, I'd avoid any transformations to options in this PR.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);

Review Comment:
   What if some older snapshots have another set of identifier fields? We don't have to support it but I wonder whether we can validate in the reader that all snapshots that are being scanned have the expected identifier columns or those are undefined. Cause if we use a set of identifier columns and it is different from the real ones, it will become a problem.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085584919


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field

Review Comment:
   needed => used?
   
   Identifier columns are used for determining whether an insert and delete record refer to the same row. If the two records share the same values for the identity columns they are considered to be before and after states of the same row.
   
   ?
   
   



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085571355


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are

Review Comment:
   pair of a delete row and an insert row



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   +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] hililiwei commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   Yes, if we're gonna store it, then i'd prefer a flink like approcach.
   ```
       /** Insertion operation. */
       INSERT("+I", (byte) 0),
   ```
   



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454164075

   retest this please
   


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454507241

   Thanks, @flyrain! I am excited to test this out in real 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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124877625


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even

Review Comment:
   It is still accurate. The default behavior is that NOT computing updates, but removing carryovers. I can change the command to this, so that it is more clear we test the default behavior here.
   ```
   "CALL %s.system.create_changelog_view(table => '%s')",
   ```



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085623802


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);

Review Comment:
   We could be checking earlier whether or not the catalog specified by this view name is allowed to create views



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1117586033


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);

Review Comment:
   removed in the next commit



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1117587181


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable

Review Comment:
   Made the change per new logic in the next commit.



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1440958204

   Resolved comments. Ready for another look. cc @aokolnychyi @RussellSpitzer 


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112406148


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestGenerateChangesProcedure.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.spark.extensions;
+
+import java.time.LocalDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestGenerateChangesProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestGenerateChangesProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.generate_changes("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "table_change_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    String beginning = LocalDateTime.now().toString();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    String afterFirstInsert = LocalDateTime.now().toString();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    String afterInsertOverwrite = LocalDateTime.now().toString();
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', "
+                + "options => map('%s', TIMESTAMP '%s','%s', TIMESTAMP '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', "
+                + "options => map('%s', TIMESTAMP '%s', '%s', TIMESTAMP '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(" + "remove_carryovers => false," + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', identifier_columns => 'id')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql("CALL %s.system.generate_changes(table => '%s')", catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(

Review Comment:
   Yes, it will. I also made the change that computing update is off by default. So that user has to explicitly set it to true to honor the the identifier columns, otherwise, they are not used. 



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      if (hasIdentifierColumns(args)) {
+        String[] identifierColumns = args.getString(5).split(",");
+
+        if (identifierColumns.length > 0) {
+          Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+          df = transform(df, repartitionColumns);
+        }
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);

Review Comment:
   Reused the same changelog iterator for removing carry-over rows only. I think we can optimize it here, for example, using the windows function. WDYT?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   I don't think I understand why we would need the previous row and the next row. If we are iterating over rows, then the current will become the previous, so we should only look forward or backward right?



-- 
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 #6012: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   Should we make it optional since it is a heavy operation?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Hi @hililiwei, added a new test case `testUpdateWithFilter`, I have verified the predicate on partition columns filters out data files at the planning phase, for predicate on non-partition columns, Spark will filer the row in memory.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1104926666


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   Hi @aokolnychyi, could you elaborate a bit? We pass the view name in the output row. I assume it is the same no matter user gives the view name or procedure gives a default 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] RussellSpitzer commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085622525


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {

Review Comment:
   basically
   If Idenifier {
   
   }
   
   if carryover {
   
   }



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454657153

   Thanks a lot @aokolnychyi! I will address these comments in a followup PR. It is a milestone. I'm also excited to see how people use it. Thanks everybody for the review, @RussellSpitzer @chenjunjiedada @hililiwei @rdblue!


-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454079500

   Thanks a lot for the review @aokolnychyi. Resolved all of them and ready for another look.


-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454281169

   retest this please


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1451383325

   I also added this to our 1.2 milestone. I think we should be able to merge it tomorrow.
   Thanks for making this happen, @flyrain!


-- 
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] hililiwei commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   When I want to view the table changes, sometimes I want to specify the filter key and time range, especially if I need to locate some problem\bug.  Can we support filtering conditions?



##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   How about `UPDATE_BEFORE` `UPDATE_AFTER`,they're a little bit more intuitive 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] aokolnychyi commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),

Review Comment:
   I am a bit worried about the number of parameters to configure boundaries. What if we replaced all of them with generic `options` and would pass those options along when loading `DataFrame`? Then instead of determining what snapshots match our timestamp range in the procedure, we would do that when scanning the changelog table. That way, users would be able to use timestamp boundaries not only via procedure but also via `DataFrame`. Right now, we only support snapshot ID boundaries.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Each snapshot schema may have a list of identifier fields. Can we use those if set and make this list a fallback if real identifier fields are not known?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   Is there another algorithm we can consider that would make it cheaper? Will something like this work?
   
   ```
   - Load DELETEs and INSERTs as a DF
   - Repartition the DF by primary key, _change_ordinal and locally sort by primary key, _change_ordinal, _operation_type
   - Call mapPartitions with a closure that would look at the previous, current and next rows
     - If the previous, current, next row keys are different, output the current row as-is
     - If the next row key is same, the current row must be DELETE and the next row must be INSERT (if not -> exception)
         - If other columns beyond the key are same, it is a copied over row
             - Output null if unchanged rows should be ignored
             - Output the current row as-is if all rows should be produced 
         - If other columns beyond key are different, it is an update
             - Output the current row as pre-update
     - If the previous row key is same as the current one, the current row must be INSERT and the previous row must be DELETE
         - If other columns beyond the key are same, it is a copied over row
             - Output null if unchanged rows should be ignored
             - Output the current row as-is if all rows should be produced 
         - If other columns beyond key are different, it is an update
             - Output the current row as post-update
   ```
   
   That would require reading the changes only once, doing a single hash-based shuffle to co-locate rows for the same key and change ordinal, keeping  at most 3 rows in memory at a time. Seems fairly cheap?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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

   Let me take a look today.


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   Made the change, could you take a look?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1122632629


##########
spark/v3.3/spark-extensions/src/main/antlr/org.apache.spark.sql.catalyst.parser.extensions/IcebergSqlExtensions.g4:
##########
@@ -173,6 +174,10 @@ stringMap
     : MAP '(' constant (',' constant)* ')'
     ;
 
+stringArray

Review Comment:
   Looks good.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.

Review Comment:
   Looks accurate now, thanks for updating!



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes

Review Comment:
   nit: What about `load insert and deletes from the changelog table`?



##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(2, "e", 12, DELETE, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data, _change_type", viewName));
+  }
+
+  @Before
+  public void setupTable() {
+    sql("CREATE TABLE %s (id INT, data STRING) USING iceberg", tableName);
+    sql("ALTER TABLE %s SET TBLPROPERTIES ('format-version'='%d')", tableName, 1);
+    sql("ALTER TABLE %s ADD PARTITION FIELD data", tableName);
+  }
+
+  @After
+  public void removeTables() {

Review Comment:
   nit: We usually have these init methods at the top.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {

Review Comment:
   I'd make this return `Identifier` and accept identifiers in `loadTable`. That would better align with the existing code in procedures.
   
   ```
   Identifier changelogTableIdent = changelogTableIdent(tableIdent);
   Dataset<Row> df = loadTable(changelogTableIdent, options(args));
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {

Review Comment:
   nit: Just `options`?



##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();

Review Comment:
   Why do we explicitly call this if we have `After` method to clean up tables? Is it because we always create a default table first? If so, can we remove the `Before` init method and just call a correct create method in each test?



##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even

Review Comment:
   Is this comment accurate? I thought we were supposed to keep carryovers in this case.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {

Review Comment:
   This seems fairly generic. What about accepting `Identifier` and moving this to `BaseProcedure`?
   
   ```
   protected Dataset<Row> loadTable(Identifier tableIdent, Map<String, String> options) {
     String tableName = Spark3Util.quotedFullIdentifier(tableCatalog().name(), tableIdent);
     return spark().read().options(options).table(tableName);
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {

Review Comment:
   nit: What about calling this `shouldComputeUpdateImages` and adding `computeUpdateImages`? I like what you did for carryovers.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull

Review Comment:
   What are these annotations? I don't think we ever used them before.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {

Review Comment:
   nit: What about renaming this as `shouldRemoveCarryoverRows` to distinguish from the method that actually removes carryovers? Sorry I overlooked it earlier.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {

Review Comment:
   What about simplifying like this?
   
   ```
   // If the identifier columns are set, we compute pre/post update images by default.
   return !args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL);
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)

Review Comment:
   What about this? I like ternary operators but Spotless formats them in a weird way if they require multiple lines.
   ```
   return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL) || args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getRepartitionExpr(Dataset<Row> df, String[] identifiers) {
+    Column[] repartitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        repartitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {

Review Comment:
   I am not sure there is a lot of value in intercepting exceptions in such cases. Spark would throw a fairly good exception and would list a set of available columns. Let's just use that. I feel this would simplify this block without impacting user experience.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull

Review Comment:
   Same question about the annotation.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {

Review Comment:
   What about calling it `applyChangelogIterator` or something?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getRepartitionExpr(Dataset<Row> df, String[] identifiers) {

Review Comment:
   If we decide to add `computeUpdateImages`, I would put this logic there directly, like you did for carryovers.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;

Review Comment:
   We already pass `ident.name()` to this method. Instead of checking for dots in the name, I think we can use the approach from the snippet above and escape the name using backticks.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {

Review Comment:
   What about having if/else instead of an extra var and ternary operator above?
   
   ```
   if (args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)) {
     return String.format("`%s_changes`", tableName);
   } else {
     return args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {

Review Comment:
   I think it should be if/else. If someone provides empty identifier columns, we should complain.
   
   ```
   if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
     return ...;
   } else {
     return ...;
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));

Review Comment:
   nit: Use `schema` var defined above?



-- 
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 merged pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi merged PR #6012:
URL: https://github.com/apache/iceberg/pull/6012


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java:
##########
@@ -53,6 +53,7 @@ private static Map<String, Supplier<ProcedureBuilder>> initProcedureBuilders() {
     mapBuilder.put("ancestors_of", AncestorsOfProcedure::builder);
     mapBuilder.put("register_table", RegisterTableProcedure::builder);
     mapBuilder.put("publish_changes", PublishChangesProcedure::builder);
+    mapBuilder.put("generate_changes", GenerateChangesProcedure::builder);

Review Comment:
   Other options like `register_change_view` `create_changelog_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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112394203


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);

Review Comment:
   Combined with comment https://github.com/apache/iceberg/pull/6012#discussion_r1084604884, we could push it down to reader to validate the identifier columns of each 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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   Thanks @aokolnychyi for the suggestion. Make sense to shuffle once. Agreed with @rdblue, to just look forward should be good, no need to search bidirectionally. Will make the change accordingly. 



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

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

   retest this please


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1399044913

   Getting to this PR soon.


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085549065


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestGenerateChangesProcedure.java:
##########
@@ -0,0 +1,452 @@
+/*
+ * 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.spark.extensions;
+
+import java.time.LocalDateTime;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestGenerateChangesProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestGenerateChangesProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.generate_changes("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "table_change_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    String beginning = LocalDateTime.now().toString();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    String afterFirstInsert = LocalDateTime.now().toString();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    String afterInsertOverwrite = LocalDateTime.now().toString();
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', "
+                + "options => map('%s', TIMESTAMP '%s','%s', TIMESTAMP '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', "
+                + "options => map('%s', TIMESTAMP '%s', '%s', TIMESTAMP '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(" + "remove_carryovers => false," + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.generate_changes(table => '%s', identifier_columns => 'id')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql("CALL %s.system.generate_changes(table => '%s')", catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(

Review Comment:
   Would this output be different without the identifier columns?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1115065756


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    // no need to validate the read options here since the reader will validate them
+    DataFrameReader reader = spark().read();
+    reader.options(readOptions(args));
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                if (k.toString().equals(SparkReadOptions.START_TIMESTAMP)
+                    || k.toString().equals(SparkReadOptions.END_TIMESTAMP)) {
+                  options.put(k.toString(), toMillis(v.toString()));

Review Comment:
   Removed the conversion. To support both format, will file separated PR in the reader side.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112286064


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {

Review Comment:
   good 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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1125362664


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java:
##########
@@ -144,6 +148,12 @@ protected SparkTable loadSparkTable(Identifier ident) {
     }
   }
 
+  protected Dataset<Row> loadDataSetFromTable(Identifier tableIdent, Map<String, String> options) {

Review Comment:
   Is there a shorter yet descriptive name? Like `loadRows`, `loadContent`, etc?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // load insert and deletes from the changelog table
+    Identifier changelogTableIdent = changelogTableIdent(tableIdent);
+    Dataset<Row> df = loadDataSetFromTable(changelogTableIdent, options(args));
+
+    if (shouldComputeUpdateImages(args)) {
+      df = computeUpdateImages(identifierColumns(args, tableIdent), df);
+    } else if (shouldRemoveCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> computeUpdateImages(String[] identifierColumns, Dataset<Row> df) {
+    Preconditions.checkArgument(
+        identifierColumns.length > 0,
+        "Cannot compute the update-rows because identifier columns are not set");
+
+    Column[] repartitionColumns = new Column[identifierColumns.length + 1];

Review Comment:
   nit: We sometimes call it `repartitionColumns` and sometimes `repartitionSpec`.
   I'd probably use `repartitionSpec` everywhere since it is shorter (this statement would fit on 1 line?) and matches `sortSpec` used in other methods.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/BaseProcedure.java:
##########
@@ -144,6 +148,12 @@ protected SparkTable loadSparkTable(Identifier ident) {
     }
   }
 
+  protected Dataset<Row> loadDataSetFromTable(Identifier tableIdent, Map<String, String> options) {
+    String tableName = Spark3Util.quotedFullIdentifier(tableCatalog().name(), tableIdent);
+    // no need to validate the read options here since the reader will validate them

Review Comment:
   I don't think we need this comment anymore since it is a pretty generic method now.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,266 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // load insert and deletes from the changelog table
+    Identifier changelogTableIdent = changelogTableIdent(tableIdent);
+    Dataset<Row> df = loadDataSetFromTable(changelogTableIdent, options(args));
+
+    if (shouldComputeUpdateImages(args)) {
+      df = computeUpdateImages(identifierColumns(args, tableIdent), df);
+    } else if (shouldRemoveCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> computeUpdateImages(String[] identifierColumns, Dataset<Row> df) {
+    Preconditions.checkArgument(
+        identifierColumns.length > 0,
+        "Cannot compute the update-rows because identifier columns are not set");

Review Comment:
   nit: `update-rows` -> `update images`?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   I agree with can pre-compute the required state by just looking back.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("orphan_file_location", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns == null || identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    DataFrameReader reader = spark().read();
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    if (!args.isNullAt(1)) {
+      long startSnapshotId = args.getLong(1);
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, startSnapshotId);
+    }
+
+    if (!args.isNullAt(2)) {
+      long endSnapshotId = args.getLong(2);
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, endSnapshotId);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+
+    Dataset<Row> dfWithUpdate =
+        df.withColumn("count", functions.count("*").over(Window.partitionBy(partitionSpec)))
+            .withColumn(
+                "rank",
+                functions
+                    .rank()
+                    .over(
+                        Window.partitionBy(partitionSpec)
+                            .orderBy(MetadataColumns.CHANGE_TYPE.name())));
+
+    Dataset<Row> preImageDf =
+        dfWithUpdate
+            .filter("rank = 1")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_PREIMAGE.name()));
+
+    Dataset<Row> postImageDf =
+        dfWithUpdate
+            .filter("rank = 2")
+            .filter("count = 2")
+            .drop("rank", "count")
+            .withColumn(
+                MetadataColumns.CHANGE_TYPE.name(),
+                functions.lit(ChangelogOperation.UPDATE_POSTIMAGE.name()));
+
+    // remove the carry-over rows
+    Dataset<Row> dfWithoutCarryOver = removeCarryOvers(preImageDf.union(postImageDf));

Review Comment:
   I agree with can pre-compute the required state by just looking ahead.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),

Review Comment:
   Changed to `options` in the procedure. Will add the timestamp range in another 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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   It would be nice to rely on `identifierFieldIds`, but can we? I might miss something, but I didn't see `update` or `merge` relies on `identifierFieldIds`. The update condition is still from user's input of SQL input, specifically, the `on` clause of `merge`. In that sense, the cdc procedure may not rely on `identifierFieldIds` 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] chenjunjiedada commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   I think we can add some doc for the detail, shorter string brings better performance in case we store row kind info into files and performs comparing row by row 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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Thanks for the suggestion. Makes sense to add a time range. Filtering should work out-of-box, will double check.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import static org.apache.iceberg.ChangelogOperation.DELETE;
+import static org.apache.iceberg.ChangelogOperation.INSERT;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    Long[] snapshotIds = getSnapshotIds(tableName, args);
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    DataFrameReader reader = spark().read();
+    if (snapshotIds[0] != null) {
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, snapshotIds[0]);
+    }
+
+    if (snapshotIds[1] != null) {
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, snapshotIds[1]);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private Long[] getSnapshotIds(String tableName, InternalRow args) {
+    Long[] snapshotIds = new Long[] {null, null};
+
+    Long startTimestamp = args.isNullAt(5) ? null : DateTimeUtil.microsToMillis(args.getLong(5));
+    Long endTimestamp = args.isNullAt(6) ? null : DateTimeUtil.microsToMillis(args.getLong(6));
+
+    if (startTimestamp == null && endTimestamp == null) {
+      snapshotIds[0] = args.isNullAt(1) ? null : args.getLong(1);
+      snapshotIds[1] = args.isNullAt(2) ? null : args.getLong(2);
+    } else {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      Snapshot[] snapshots = snapshotsFromTimestamp(startTimestamp, endTimestamp, table);
+
+      if (snapshots != null) {
+        snapshotIds[0] = snapshots[0].parentId();
+        snapshotIds[1] = snapshots[1].snapshotId();
+      }
+    }
+    return snapshotIds;
+  }
+
+  private Snapshot[] snapshotsFromTimestamp(Long startTimestamp, Long endTimestamp, Table table) {
+    Snapshot[] snapshots = new Snapshot[] {null, null};
+
+    if (startTimestamp != null && endTimestamp != null && startTimestamp > endTimestamp) {
+      throw new IllegalArgumentException(
+          "Start timestamp must be less than or equal to end timestamp");
+    }
+
+    if (startTimestamp == null) {
+      snapshots[0] = SnapshotUtil.oldestAncestor(table);
+    } else {
+      snapshots[0] = SnapshotUtil.oldestAncestorAfter(table, startTimestamp);
+    }
+
+    if (endTimestamp == null) {
+      snapshots[1] = table.currentSnapshot();
+    } else {
+      snapshots[1] = table.snapshot(SnapshotUtil.snapshotIdAsOfTime(table, endTimestamp));
+    }
+
+    if (snapshots[0] == null || snapshots[1] == null) {
+      return null;
+    }
+
+    return snapshots;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+    Column[] sortSpec = sortSpec(df, partitionSpec);
+
+    int changeTypeIdx = df.schema().fieldIndex(MetadataColumns.CHANGE_TYPE.name());
+    List<Integer> partitionIdx =
+        Arrays.stream(partitionSpec)
+            .map(column -> df.schema().fieldIndex(column.toString()))
+            .collect(Collectors.toList());
+
+    return df.repartition(partitionSpec)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            processRowsWithinTask(changeTypeIdx, partitionIdx), RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getPartitionSpec(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        partitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            String.format("Identifier column '%s' does not exist in the table", identifiers[i]), e);
+      }
+    }
+    partitionSpec[partitionSpec.length - 1] = df.col(MetadataColumns.CHANGE_ORDINAL.name());
+    return partitionSpec;
+  }
+
+  @NotNull
+  private static Column[] sortSpec(Dataset<Row> df, Column[] partitionSpec) {
+    Column[] sortSpec = new Column[partitionSpec.length + 1];
+    System.arraycopy(partitionSpec, 0, sortSpec, 0, partitionSpec.length);
+    sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name());
+    return sortSpec;
+  }
+
+  private static MapPartitionsFunction<Row, Row> processRowsWithinTask(
+      int changeTypeIndex, List<Integer> partitionIdx) {
+    return rowIterator -> {
+      Iterator<Row> iterator =
+          new Iterator<Row>() {
+            private Row nextRow = null;
+
+            @Override
+            public boolean hasNext() {
+              if (nextRow != null) {
+                return true;
+              }
+              return rowIterator.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              // if the next row is cached and changed, return it directly
+              if (nextRow != null
+                  && !nextRow.getString(changeTypeIndex).equals(DELETE.name())
+                  && !nextRow.getString(changeTypeIndex).equals(INSERT.name())) {
+                Row row = nextRow;
+                nextRow = null;
+                return row;
+              }
+
+              Row currentRow = currentRow();
+
+              if (rowIterator.hasNext()) {
+                GenericRowWithSchema nextRow = (GenericRowWithSchema) rowIterator.next();
+
+                if (withinPartition(currentRow, nextRow)
+                    && currentRow.getString(changeTypeIndex).equals(DELETE.name())
+                    && nextRow.getString(changeTypeIndex).equals(INSERT.name())) {

Review Comment:
   It is rare that two rows are from different partitions, but it can happen, especially AQE can merge different partitions together. It is hard to add a unit test for this case though.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112404844


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");

Review Comment:
   Made 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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112286064


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {

Review Comment:
   good idea. Fixed in the new commit.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124904991


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull

Review Comment:
   It comes from IntelliJ refactor. I'm OK with either keeping or removing it. Will remove it in the next commit.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124911280


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even

Review Comment:
   But this test calls `remove_carryovers = false` and the carryovers are not removed as far as I see in the check below?



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "RussellSpitzer (via GitHub)" <gi...@apache.org>.
RussellSpitzer commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1085603592


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   We can add in a "precondition" "catalog blah is does not support views"



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   It would be nice to rely on `identifierFieldIds`, but can we? I might miss something, but I didn't see `update` or `merge` relies on `identifierFieldIds`. The update condition is still from user's input, specifically, the `on` clause of a `merge` command. In that sense, the cdc procedure may not rely on `identifierFieldIds` 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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   This PR was updated to use `UPDATE_BEFORE` and `UPDATE_AFTER` so I am resolving this thread. We can't use short versions in Spark as the `changes` table has been already released.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import static org.apache.iceberg.ChangelogOperation.DELETE;
+import static org.apache.iceberg.ChangelogOperation.INSERT;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+        ProcedureParameter.optional("start_timestamp", DataTypes.TimestampType),
+        ProcedureParameter.optional("end_timestamp", DataTypes.TimestampType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // Compute the pre-image and post-images if the identifier columns are provided.
+    if (!args.isNullAt(4)) {
+      String[] identifierColumns = args.getString(4).split(",");
+      if (identifierColumns.length > 0) {
+        df = withUpdate(df, identifierColumns);
+      }
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    Long[] snapshotIds = getSnapshotIds(tableName, args);
+
+    // we don't have to validate the snapshot ids here because the reader will do it for us.
+    DataFrameReader reader = spark().read();
+    if (snapshotIds[0] != null) {
+      reader = reader.option(SparkReadOptions.START_SNAPSHOT_ID, snapshotIds[0]);
+    }
+
+    if (snapshotIds[1] != null) {
+      reader = reader.option(SparkReadOptions.END_SNAPSHOT_ID, snapshotIds[1]);
+    }
+
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  @NotNull
+  private Long[] getSnapshotIds(String tableName, InternalRow args) {
+    Long[] snapshotIds = new Long[] {null, null};
+
+    Long startTimestamp = args.isNullAt(5) ? null : DateTimeUtil.microsToMillis(args.getLong(5));
+    Long endTimestamp = args.isNullAt(6) ? null : DateTimeUtil.microsToMillis(args.getLong(6));
+
+    if (startTimestamp == null && endTimestamp == null) {
+      snapshotIds[0] = args.isNullAt(1) ? null : args.getLong(1);
+      snapshotIds[1] = args.isNullAt(2) ? null : args.getLong(2);
+    } else {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      Snapshot[] snapshots = snapshotsFromTimestamp(startTimestamp, endTimestamp, table);
+
+      if (snapshots != null) {
+        snapshotIds[0] = snapshots[0].parentId();
+        snapshotIds[1] = snapshots[1].snapshotId();
+      }
+    }
+    return snapshotIds;
+  }
+
+  private Snapshot[] snapshotsFromTimestamp(Long startTimestamp, Long endTimestamp, Table table) {
+    Snapshot[] snapshots = new Snapshot[] {null, null};
+
+    if (startTimestamp != null && endTimestamp != null && startTimestamp > endTimestamp) {
+      throw new IllegalArgumentException(
+          "Start timestamp must be less than or equal to end timestamp");
+    }
+
+    if (startTimestamp == null) {
+      snapshots[0] = SnapshotUtil.oldestAncestor(table);
+    } else {
+      snapshots[0] = SnapshotUtil.oldestAncestorAfter(table, startTimestamp);
+    }
+
+    if (endTimestamp == null) {
+      snapshots[1] = table.currentSnapshot();
+    } else {
+      snapshots[1] = table.snapshot(SnapshotUtil.snapshotIdAsOfTime(table, endTimestamp));
+    }
+
+    if (snapshots[0] == null || snapshots[1] == null) {
+      return null;
+    }
+
+    return snapshots;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(3) ? null : args.getString(3);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> withUpdate(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = getPartitionSpec(df, identifiers);
+    Column[] sortSpec = sortSpec(df, partitionSpec);
+
+    int changeTypeIdx = df.schema().fieldIndex(MetadataColumns.CHANGE_TYPE.name());
+    List<Integer> partitionIdx =
+        Arrays.stream(partitionSpec)
+            .map(column -> df.schema().fieldIndex(column.toString()))
+            .collect(Collectors.toList());
+
+    return df.repartition(partitionSpec)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            processRowsWithinTask(changeTypeIdx, partitionIdx), RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getPartitionSpec(Dataset<Row> df, String[] identifiers) {
+    Column[] partitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        partitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            String.format("Identifier column '%s' does not exist in the table", identifiers[i]), e);
+      }
+    }
+    partitionSpec[partitionSpec.length - 1] = df.col(MetadataColumns.CHANGE_ORDINAL.name());
+    return partitionSpec;
+  }
+
+  @NotNull
+  private static Column[] sortSpec(Dataset<Row> df, Column[] partitionSpec) {
+    Column[] sortSpec = new Column[partitionSpec.length + 1];
+    System.arraycopy(partitionSpec, 0, sortSpec, 0, partitionSpec.length);
+    sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name());
+    return sortSpec;
+  }
+
+  private static MapPartitionsFunction<Row, Row> processRowsWithinTask(
+      int changeTypeIndex, List<Integer> partitionIdx) {
+    return rowIterator -> {
+      Iterator<Row> iterator =
+          new Iterator<Row>() {
+            private Row nextRow = null;
+
+            @Override
+            public boolean hasNext() {
+              if (nextRow != null) {
+                return true;
+              }
+              return rowIterator.hasNext();
+            }
+
+            @Override
+            public Row next() {
+              // if the next row is cached and changed, return it directly
+              if (nextRow != null
+                  && !nextRow.getString(changeTypeIndex).equals(DELETE.name())
+                  && !nextRow.getString(changeTypeIndex).equals(INSERT.name())) {
+                Row row = nextRow;
+                nextRow = null;
+                return row;
+              }
+
+              Row currentRow = currentRow();
+
+              if (rowIterator.hasNext()) {
+                GenericRowWithSchema nextRow = (GenericRowWithSchema) rowIterator.next();
+
+                if (withinPartition(currentRow, nextRow)
+                    && currentRow.getString(changeTypeIndex).equals(DELETE.name())
+                    && nextRow.getString(changeTypeIndex).equals(INSERT.name())) {

Review Comment:
   Done and added unit tests.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,336 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Objects;
+import java.util.stream.Collectors;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Iterators;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.RowFactory;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.catalyst.expressions.GenericInternalRow;
+import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {

Review Comment:
   Done



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

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

   Ready for another look. cc @RussellSpitzer @szehon-ho @aokolnychyi 


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {

Review Comment:
   can we pull the if (removeCarryoverRow) outside of this if statement?  



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    // no need to validate the read options here since the reader will validate them
+    DataFrameReader reader = spark().read();
+    reader.options(readOptions(args));
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                if (k.toString().equals(SparkReadOptions.START_TIMESTAMP)
+                    || k.toString().equals(SparkReadOptions.END_TIMESTAMP)) {
+                  options.put(k.toString(), toMillis(v.toString()));

Review Comment:
   The read option only accepts the string of milliseconds, not the string of timestamp like `2019-02-08 03:29:51.215`. Here is an example of read option.
   ```
   // time travel to October 26, 1986 at 01:21:00
   spark.read
       .option("as-of-timestamp", "499162860000")
       .format("iceberg")
       .load("path/to/table")
   ```



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      if (hasIdentifierColumns(args)) {
+        String[] identifierColumns = args.getString(5).split(",");
+
+        if (identifierColumns.length > 0) {
+          Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+          df = transform(df, repartitionColumns);
+        }
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);

Review Comment:
   The motivation of building the changelog iterator is to combine two operation together in one pass. But if there is only one operation, a window function seems fit better.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1084807856


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    // no need to validate the read options here since the reader will validate them
+    DataFrameReader reader = spark().read();
+    reader.options(readOptions(args));
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                if (k.toString().equals(SparkReadOptions.START_TIMESTAMP)
+                    || k.toString().equals(SparkReadOptions.END_TIMESTAMP)) {
+                  options.put(k.toString(), toMillis(v.toString()));
+                } else {
+                  options.put(k.toString(), v.toString());
+                }
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  private static String toMillis(String timestamp) {
+    return String.valueOf(Timestamp.valueOf(timestamp).getTime());
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(1) ? null : args.getString(1);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+
+    int changeTypeIdx = df.schema().fieldIndex(MetadataColumns.CHANGE_TYPE.name());
+
+    List<Integer> repartitionIdx =
+        Arrays.stream(repartitionColumns)
+            .map(column -> df.schema().fieldIndex(column.toString()))
+            .collect(Collectors.toList());
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator ->
+                    ChangelogIterator.iterator(rowIterator, changeTypeIdx, repartitionIdx),

Review Comment:
   I had a few questions about `ChangelogIterator`, which I left on #6344.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112266092


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field

Review Comment:
   Made the change accordingly.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124950226


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)
+        ? true
+        : args.getBoolean(REMOVE_CARRYOVERS_ORDINAL);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args) {
+    String[] identifierColumns = new String[0];
+
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      identifierColumns =
+          Arrays.stream(args.getArray(IDENTIFIER_COLUMNS_ORDINAL).array())
+              .map(column -> column.toString())
+              .toArray(String[]::new);
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent =
+          toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark().read().options(readOptions).table(tableName);
+  }
+
+  private String changelogTableName(Identifier tableIdent) {
+    List<String> namespace = Lists.newArrayList();
+    namespace.addAll(Arrays.asList(tableIdent.namespace()));
+    namespace.add(tableIdent.name());
+    Identifier changelogTableIdent =
+        Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
+    return Spark3Util.quotedFullIdentifier(tableCatalog().name(), changelogTableIdent);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(OPTIONS_ORDINAL)) {
+      args.getMap(OPTIONS_ORDINAL)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName =
+        args.isNullAt(CHANGELOG_VIEW_NAME_ORDINAL)
+            ? null
+            : args.getString(CHANGELOG_VIEW_NAME_ORDINAL);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getRepartitionExpr(Dataset<Row> df, String[] identifiers) {
+    Column[] repartitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        repartitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {

Review Comment:
   Good to know. Will make the change in the next commit



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124939391


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)

Review Comment:
   I made the change as following. More lines but easier to understand. Maybe I'm dumb, but I always need a little more time to understand a expression like `a || b`. 
   ```
      if (args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)) {
         return true;
       } else {
         return args.getBoolean(REMOVE_CARRYOVERS_ORDINAL)
      }
   ```



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124970579


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even

Review Comment:
   oh, sorry, I'm looking at a different test. I have removed the comment in the new commit. Thanks for catching it.



-- 
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 #6012: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   Open for naming suggestions



-- 
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 #6012: Add a procedure to generate table changes

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


##########
api/src/main/java/org/apache/iceberg/ChangelogOperation.java:
##########
@@ -21,5 +21,7 @@
 /** An enum representing possible operations in a changelog. */
 public enum ChangelogOperation {
   INSERT,
-  DELETE
+  DELETE,
+  UPDATE_PREIMAGE,
+  UPDATE_POSTIMAGE

Review Comment:
   I also like the idea of a short version of CDC record Type like (I,D, -U, +U). What do you think?
   1. Insert(I), indicating an inserted row
   2. Delete(D), indicating a deleted row
   3. Pre-update(-U), indicating the original value of an updated row
   4. Post-update(+U), indicating the new value of an updated row
   



-- 
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 pull request #6012: Add a procedure to generate table changes

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

   The test failure is not related.


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,210 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   @hililiwei, added support for timestamp-based query, check `testTimestampsBasedQuery` out.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1112262090


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),

Review Comment:
   Got it by checking this comment https://github.com/apache/iceberg/pull/6012#discussion_r1085623802. Let me put a precondition check on the catalog 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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "aokolnychyi (via GitHub)" <gi...@apache.org>.
aokolnychyi commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1116278037


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable

Review Comment:
   How accurate is this description now after recent changes?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns

Review Comment:
   nit: What about `pre/post update images` instead of `update-row` in all places?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Shouldn't this be an array of strings? That way, we don't have to deal with escaping and parsing.
   We can also define a helper variable for it in `BaseProcedure`, just like we have `STRING_MAP` today.
   
   ```
   protected static final DataType STRING_ARRAY = DataTypes.createArrayType(DataTypes.StringType);
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.

Review Comment:
   nit: `iterator` -> `procedure`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);

Review Comment:
   We can't use the table name as string because procedures resolve tables differently. We should follow what we do in other procedures.
   
   ```
   Identifier tableIdent = toIdentifier(args.getString(0), PARAMETERS[0].name());
   ```
   
   Under the hood, it will default the catalog to the catalog for which this procedure is invoked, which may be different from the default Spark catalog in the session.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark()
+        .read()
+        .options(readOptions)
+        .table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {

Review Comment:
   Let me think about defaulting the view name tomorrow.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {

Review Comment:
   We will need an identifier in this method so that the resolution works correctly.
   We could have something like this.
   
   ```
   private Dataset<Row> changelogDF(Identifier tableIdent, Map<String, String> options) {
     Identifier changelogTableIdent = changelogTableIdent(tableIdent);
     return loadTable(changelogTableIdent, options);
   }
   
   private Identifier changelogTableIdent(Identifier tableIdent) {
     List<String> namespace = Lists.newArrayList();
     namespace.addAll(Arrays.asList(tableIdent.namespace()));
     namespace.add(tableIdent.name());
     return Identifier.of(namespace.toArray(new String[0]), SparkChangelogTable.TABLE_NAME);
   }
   ```
   
   And then add the following method to `BaseProcedure`:
   
   ```
   protected Dataset<Row> loadTable(Identifier tableIdent, Map<String, String> options) {
     String tableName = Spark3Util.quotedFullIdentifier(tableCatalog().name(), tableIdent);
     return spark().read().options(options).table(tableName);
   }
   ```



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,

Review Comment:
   nit: `change-log` -> `changelog`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input

Review Comment:
   nit: Identifier fields are not set in the table properties. They are set in the schema.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).

Review Comment:
   nit: `update-row` -> `pre/post update images`



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input

Review Comment:
   nit: Just `identifier field IDs`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {

Review Comment:
   What about calling it `CreateChangelogViewProcedure`? We kind of provide a real changelog with update images.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query

Review Comment:
   nit: This seems to be fairly obvious, do we need this comment?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);

Review Comment:
   Is this being used?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {

Review Comment:
   Can you explore the idea of using native `dropDuplicates` operation instead of applying the iterator? At least, this should avoid the conversion from the internal to the public row representation, which is required by the iterator at the moment.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark()
+        .read()
+        .options(readOptions)
+        .table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(1) ? null : args.getString(1);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {

Review Comment:
   If we decide to use `dropDuplicates` to remove carryovers, this may have a more specific name.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);

Review Comment:
   What about defining some static vars to give meaning to these numbers?
   
   ```
   private static final int COMPUTE_UPDATES_ORDINAL = 3;
   ```
   
   We do that in Spark functions. I will also think about a better approach to handle parsing of args.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {

Review Comment:
   It is a good idea to compute update images if the user provided identifier columns but only if the flag is not set to false. If the flag to compute update images is explicitly set to false, we should respect it.



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);

Review Comment:
   nit: What about defining a method `removeCarryovers(args)` like we have for other args?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {

Review Comment:
   nit: What about `computeUpdateImages`?



##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangeViewProcedure.java:
##########
@@ -0,0 +1,267 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of a delete row and an insert row. Identifier columns
+ * are used for determining whether an insert and a delete record refer to the same row. If the two
+ * records share the same values for the identity columns they are considered to be before and after
+ * states of the same row. You can either set Identifier Field IDs as the table properties or input
+ * them as the procedure parameters. Here is an example of update-row with an identifier column(id).
+ * A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangeViewProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(CreateChangeViewProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangeViewProcedure>() {
+      @Override
+      protected CreateChangeViewProcedure doBuild() {
+        return new CreateChangeViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangeViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, readOptions(args));
+
+    // compute remove carry-over rows by default
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow(args)) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdatedRow(InternalRow args) {
+    if (!args.isNullAt(5)) {
+      return true;
+    }
+
+    return args.isNullAt(3) ? false : args.getBoolean(3);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, Map<String, String> readOptions) {
+    // no need to validate the read options here since the reader will validate them
+    return spark()
+        .read()
+        .options(readOptions)
+        .table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                options.put(k.toString(), v.toString());
+                return BoxedUnit.UNIT;
+              });
+    }
+
+    return options;
+  }
+
+  @NotNull
+  private static String viewName(InternalRow args, String tableName) {
+    String viewName = args.isNullAt(1) ? null : args.getString(1);
+    if (viewName == null) {
+      String shortTableName =
+          tableName.contains(".") ? tableName.substring(tableName.lastIndexOf(".") + 1) : tableName;
+      viewName = shortTableName + "_changes";
+    }
+    return viewName;
+  }
+
+  private Dataset<Row> transform(Dataset<Row> df, Column[] repartitionColumns) {
+    Column[] sortSpec = sortSpec(df, repartitionColumns);
+    StructType schema = df.schema();
+    String[] identifierFields =
+        Arrays.stream(repartitionColumns).map(Column::toString).toArray(String[]::new);
+
+    return df.repartition(repartitionColumns)
+        .sortWithinPartitions(sortSpec)
+        .mapPartitions(
+            (MapPartitionsFunction<Row, Row>)
+                rowIterator -> ChangelogIterator.create(rowIterator, schema, identifierFields),
+            RowEncoder.apply(df.schema()));
+  }
+
+  @NotNull
+  private static Column[] getRepartitionExpr(Dataset<Row> df, String[] identifiers) {
+    Column[] repartitionSpec = new Column[identifiers.length + 1];
+    for (int i = 0; i < identifiers.length; i++) {
+      try {
+        repartitionSpec[i] = df.col(identifiers[i]);
+      } catch (Exception e) {
+        throw new IllegalArgumentException(
+            String.format("Identifier column '%s' does not exist in the table", identifiers[i]), e);
+      }
+    }
+    repartitionSpec[repartitionSpec.length - 1] = df.col(MetadataColumns.CHANGE_ORDINAL.name());
+    return repartitionSpec;
+  }
+
+  @NotNull
+  private static Column[] sortSpec(Dataset<Row> df, Column[] repartitionSpec) {
+    Column[] sortSpec = new Column[repartitionSpec.length + 1];
+    System.arraycopy(repartitionSpec, 0, sortSpec, 0, repartitionSpec.length);
+    sortSpec[sortSpec.length - 1] = df.col(MetadataColumns.CHANGE_TYPE.name());
+    return sortSpec;
+  }
+
+  private InternalRow[] toOutputRows(String viewName) {
+    InternalRow row = newInternalRow(UTF8String.fromString(viewName));
+    return new InternalRow[] {row};
+  }
+
+  @Override
+  public String description() {
+    return "GenerateChangesProcedure";

Review Comment:
   This should be updated to reflect the new procedure 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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124884488


##########
spark/v3.3/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestCreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,457 @@
+/*
+ * 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.spark.extensions;
+
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestCreateChangelogViewProcedure extends SparkExtensionsTestBase {
+  private static final String DELETE = ChangelogOperation.DELETE.name();
+  private static final String INSERT = ChangelogOperation.INSERT.name();
+  private static final String UPDATE_BEFORE = ChangelogOperation.UPDATE_BEFORE.name();
+  private static final String UPDATE_AFTER = ChangelogOperation.UPDATE_AFTER.name();
+
+  public TestCreateChangelogViewProcedure(
+      String catalogName, String implementation, Map<String, String> config) {
+    super(catalogName, implementation, config);
+  }
+
+  @Test
+  public void testCustomizedViewName() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+
+    Table table = validationCatalog.loadTable(tableIdent);
+
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+
+    table.refresh();
+
+    Snapshot snap2 = table.currentSnapshot();
+
+    sql(
+        "CALL %s.system.create_changelog_view("
+            + "table => '%s',"
+            + "options => map('%s','%s','%s','%s'),"
+            + "changelog_view => '%s')",
+        catalogName,
+        tableName,
+        SparkReadOptions.START_SNAPSHOT_ID,
+        snap1.snapshotId(),
+        SparkReadOptions.END_SNAPSHOT_ID,
+        snap2.snapshotId(),
+        "cdc_view");
+
+    long rowCount = sql("select * from %s", "cdc_view").stream().count();
+    Assert.assertEquals(2, rowCount);
+  }
+
+  @Test
+  public void testNoSnapshotIdInput() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(" + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", viewName));
+  }
+
+  @Test
+  public void testTimestampsBasedQuery() {
+    long beginning = System.currentTimeMillis();
+
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+    long afterFirstInsert = waitUntilAfter(snap0.timestampMillis());
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    long afterInsertOverwrite = waitUntilAfter(snap2.timestampMillis());
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s','%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            beginning,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+
+    // query the timestamps starting from the second insert
+    returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', "
+                + "options => map('%s', '%s', '%s', '%s'))",
+            catalogName,
+            tableName,
+            SparkReadOptions.START_TIMESTAMP,
+            afterFirstInsert,
+            SparkReadOptions.END_TIMESTAMP,
+            afterInsertOverwrite);
+
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(-2, "b", INSERT, 1, snap2.snapshotId()),
+            row(2, "b", DELETE, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id", returns.get(0)[0]));
+  }
+
+  @Test
+  public void testWithCarryovers() {
+    sql("INSERT INTO %s VALUES (1, 'a')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap0 = table.currentSnapshot();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (-2, 'b'), (2, 'b'), (2, 'b')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "remove_carryovers => false,"
+                + "table => '%s')",
+            catalogName, tableName, "cdc_view");
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap0.snapshotId()),
+            row(2, "b", INSERT, 1, snap1.snapshotId()),
+            row(-2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", DELETE, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId()),
+            row(2, "b", INSERT, 2, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, _change_type", viewName));
+  }
+
+  @Test
+  public void testUpdate() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithIdentifierField() {
+    removeTables();
+    createTableWithIdentifierField();
+
+    sql("INSERT INTO %s VALUES (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', compute_updates => true)",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithFilter() {
+    sql("ALTER TABLE %s DROP PARTITION FIELD data", tableName);
+    sql("ALTER TABLE %s ADD PARTITION FIELD id", tableName);
+
+    sql("INSERT INTO %s VALUES (1, 'a'), (2, 'b')", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c'), (2, 'd')", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view(table => '%s', identifier_columns => array('id'))",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", INSERT, 0, snap1.snapshotId()),
+            row(2, "b", UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", UPDATE_AFTER, 1, snap2.snapshotId())),
+        // the predicate on partition columns will filter out the insert of (3, 'c') at the planning
+        // phase
+        sql("select * from %s where id != 3 order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testUpdateWithMultipleIdentifierColumns() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'),"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(2, "e", 12, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOvers() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "identifier_columns => array('id','age'), "
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, UPDATE_BEFORE, 1, snap2.snapshotId()),
+            row(2, "d", 11, UPDATE_AFTER, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testRemoveCarryOversWithoutUpdatedRows() {
+    removeTables();
+    createTableWith3Columns();
+
+    sql("INSERT INTO %s VALUES (1, 'a', 12), (2, 'b', 11), (2, 'e', 12)", tableName);
+    Table table = validationCatalog.loadTable(tableIdent);
+    Snapshot snap1 = table.currentSnapshot();
+
+    // carry-over row (2, 'e', 12)
+    sql("INSERT OVERWRITE %s VALUES (3, 'c', 13), (2, 'd', 11), (2, 'e', 12)", tableName);
+    table.refresh();
+    Snapshot snap2 = table.currentSnapshot();
+
+    List<Object[]> returns =
+        sql(
+            "CALL %s.system.create_changelog_view("
+                + "compute_updates => false,"
+                + "table => '%s')",
+            catalogName, tableName);
+
+    String viewName = (String) returns.get(0)[0];
+
+    // the carry-over rows (2, 'e', 12, 'DELETE', 1), (2, 'e', 12, 'INSERT', 1) are removed, even
+    // though update-row is not computed
+    assertEquals(
+        "Rows should match",
+        ImmutableList.of(
+            row(1, "a", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, INSERT, 0, snap1.snapshotId()),
+            row(2, "e", 12, INSERT, 0, snap1.snapshotId()),
+            row(2, "b", 11, DELETE, 1, snap2.snapshotId()),
+            row(2, "d", 11, INSERT, 1, snap2.snapshotId()),
+            row(3, "c", 13, INSERT, 1, snap2.snapshotId())),
+        sql("select * from %s order by _change_ordinal, id, data", viewName));
+  }
+
+  @Test
+  public void testNotRemoveCarryOvers() {
+    removeTables();

Review Comment:
   That's a good idea. Made the change in the next commit.



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on code in PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#discussion_r1124939391


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/CreateChangelogViewProcedure.java:
##########
@@ -0,0 +1,299 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.Lists;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.Spark3Util;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure removes the carry-over rows by default. If you want to keep them, you can set
+ * "remove_carryovers" to be false in the options.
+ *
+ * <p>The procedure doesn't compute the pre/post update images by default. If you want to compute
+ * them, you can set "compute_updates" to be true in the options.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The changelog table would report this as (id=1, data='a',
+ * op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change to the
+ * table. The procedure finds the carry-over rows and removes them from the result.
+ *
+ * <p>Pre/post update images are converted from a pair of a delete row and an insert row. Identifier
+ * columns are used for determining whether an insert and a delete record refer to the same row. If
+ * the two records share the same values for the identity columns they are considered to be before
+ * and after states of the same row. You can either set identifier fields in the table schema or
+ * input them as the procedure parameters. Here is an example of pre/post update images with an
+ * identifier column(id). A pair of a delete row and an insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as pre/post update images:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class CreateChangelogViewProcedure extends BaseProcedure {
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("changelog_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", STRING_ARRAY),
+      };
+
+  private static final int TABLE_NAME_ORDINAL = 0;
+  private static final int CHANGELOG_VIEW_NAME_ORDINAL = 1;
+  private static final int OPTIONS_ORDINAL = 2;
+  private static final int COMPUTE_UPDATES_ORDINAL = 3;
+  private static final int REMOVE_CARRYOVERS_ORDINAL = 4;
+  private static final int IDENTIFIER_COLUMNS_ORDINAL = 5;
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("changelog_view", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<CreateChangelogViewProcedure>() {
+      @Override
+      protected CreateChangelogViewProcedure doBuild() {
+        return new CreateChangelogViewProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private CreateChangelogViewProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    Identifier tableIdent =
+        toIdentifier(args.getString(TABLE_NAME_ORDINAL), PARAMETERS[TABLE_NAME_ORDINAL].name());
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(changelogTableName(tableIdent), readOptions(args));
+
+    if (computeUpdateImages(args)) {
+      String[] identifierColumns = identifierColumns(args);
+
+      Preconditions.checkArgument(
+          identifierColumns.length > 0,
+          "Cannot compute the update-rows because identifier columns are not set");
+
+      Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+      df = transform(df, repartitionColumns);
+    } else if (removeCarryoverRows(args)) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableIdent.name());
+
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private boolean computeUpdateImages(InternalRow args) {
+    if (!args.isNullAt(COMPUTE_UPDATES_ORDINAL)) {
+      return args.getBoolean(COMPUTE_UPDATES_ORDINAL);
+    }
+
+    // If the identifier columns are set, we compute pre/post update images by default.
+    if (!args.isNullAt(IDENTIFIER_COLUMNS_ORDINAL)) {
+      return true;
+    }
+
+    return false;
+  }
+
+  private boolean removeCarryoverRows(InternalRow args) {
+    return args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)

Review Comment:
   I made the change as following. More lines but easier to understand. Maybe I'm dumb, but I always need a little more time to understand a expression like `a || b`. 
   ```
      if (args.isNullAt(REMOVE_CARRYOVERS_ORDINAL)) {
         return true;
       }
       return args.getBoolean(REMOVE_CARRYOVERS_ORDINAL)
   ```



-- 
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 pull request #6012: Spark 3.3: Add a procedure to generate table changes

Posted by "flyrain (via GitHub)" <gi...@apache.org>.
flyrain commented on PR #6012:
URL: https://github.com/apache/iceberg/pull/6012#issuecomment-1454164621

   The pipeline error is not related.
   ```
   > A failure occurred while executing org.gradle.api.plugins.quality.internal.CheckstyleAction
      > An unexpected error occurred configuring and executing Checkstyle.
         > java.lang.Error: Error was thrown while processing /home/runner/work/iceberg/iceberg/spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/source/SparkPartitioningAwareScan.java
   ```


-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/SparkProcedures.java:
##########
@@ -53,6 +53,7 @@ private static Map<String, Supplier<ProcedureBuilder>> initProcedureBuilders() {
     mapBuilder.put("ancestors_of", AncestorsOfProcedure::builder);
     mapBuilder.put("register_table", RegisterTableProcedure::builder);
     mapBuilder.put("publish_changes", PublishChangesProcedure::builder);
+    mapBuilder.put("generate_changes", GenerateChangesProcedure::builder);

Review Comment:
   how about `create_change_view` `generate_change_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] flyrain commented on a diff in pull request #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,271 @@
+/*
+ * 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.spark.procedures;
+
+import java.util.Arrays;
+import java.util.UUID;
+import org.apache.iceberg.ChangelogOperation;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Snapshot;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.iceberg.util.DateTimeUtil;
+import org.apache.iceberg.util.SnapshotUtil;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.expressions.Window;
+import org.apache.spark.sql.functions;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+
+public class GenerateChangesProcedure extends BaseProcedure {
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        // the snapshot ids input are ignored when the start/end timestamps are provided
+        ProcedureParameter.optional("start_snapshot_id_exclusive", DataTypes.LongType),
+        ProcedureParameter.optional("end_snapshot_id_inclusive", DataTypes.LongType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),

Review Comment:
   Made the change per suggestion



-- 
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 #6012: Spark 3.3: Add a procedure to generate table changes

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


##########
spark/v3.3/spark/src/main/java/org/apache/iceberg/spark/procedures/GenerateChangesProcedure.java:
##########
@@ -0,0 +1,278 @@
+/*
+ * 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.spark.procedures;
+
+import java.sql.Timestamp;
+import java.util.Arrays;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.iceberg.MetadataColumns;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.relocated.com.google.common.collect.Maps;
+import org.apache.iceberg.spark.ChangelogIterator;
+import org.apache.iceberg.spark.SparkReadOptions;
+import org.apache.iceberg.spark.source.SparkChangelogTable;
+import org.apache.spark.api.java.function.MapPartitionsFunction;
+import org.apache.spark.sql.Column;
+import org.apache.spark.sql.DataFrameReader;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.catalyst.InternalRow;
+import org.apache.spark.sql.catalyst.encoders.RowEncoder;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.connector.iceberg.catalog.ProcedureParameter;
+import org.apache.spark.sql.types.DataTypes;
+import org.apache.spark.sql.types.Metadata;
+import org.apache.spark.sql.types.StructField;
+import org.apache.spark.sql.types.StructType;
+import org.apache.spark.unsafe.types.UTF8String;
+import org.jetbrains.annotations.NotNull;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A procedure that creates a view for changed rows.
+ *
+ * <p>The procedure computes update-rows and removes the carry-over rows by default. You can disable
+ * them through parameters to get better performance.
+ *
+ * <p>Carry-over rows are the result of a removal and insertion of the same row within an operation
+ * because of the copy-on-write mechanism. For example, given a file which contains row1 (id=1,
+ * data='a') and row2 (id=2, data='b'). A copy-on-write delete of row2 would require erasing this
+ * file and preserving row1 in a new file. The change-log table would report this as (id=1,
+ * data='a', op='DELETE') and (id=1, data='a', op='INSERT'), despite it not being an actual change
+ * to the table. The iterator finds the carry-over rows and removes them from the result.
+ *
+ * <p>An update-row is converted from a pair of delete row and insert row. Identifier columns are
+ * needed for identifying whether they refer to the same row. You can either set Identifier Field
+ * IDs as the table properties or input them as the procedure parameters. Here is an example of
+ * update-row with an identifier column(id). A pair of delete row and insert row with the same id:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='DELETE')
+ *   <li>(id=1, data='b', op='INSERT')
+ * </ul>
+ *
+ * <p>will be marked as update-rows:
+ *
+ * <ul>
+ *   <li>(id=1, data='a', op='UPDATE_BEFORE')
+ *   <li>(id=1, data='b', op='UPDATE_AFTER')
+ * </ul>
+ */
+public class GenerateChangesProcedure extends BaseProcedure {
+  private static final Logger LOG = LoggerFactory.getLogger(GenerateChangesProcedure.class);
+
+  private static final ProcedureParameter[] PARAMETERS =
+      new ProcedureParameter[] {
+        ProcedureParameter.required("table", DataTypes.StringType),
+        ProcedureParameter.optional("table_change_view", DataTypes.StringType),
+        ProcedureParameter.optional("options", STRING_MAP),
+        ProcedureParameter.optional("compute_updates", DataTypes.BooleanType),
+        ProcedureParameter.optional("remove_carryovers", DataTypes.BooleanType),
+        ProcedureParameter.optional("identifier_columns", DataTypes.StringType),
+      };
+
+  private static final StructType OUTPUT_TYPE =
+      new StructType(
+          new StructField[] {
+            new StructField("view_name", DataTypes.StringType, false, Metadata.empty())
+          });
+
+  public static SparkProcedures.ProcedureBuilder builder() {
+    return new BaseProcedure.Builder<GenerateChangesProcedure>() {
+      @Override
+      protected GenerateChangesProcedure doBuild() {
+        return new GenerateChangesProcedure(tableCatalog());
+      }
+    };
+  }
+
+  private GenerateChangesProcedure(TableCatalog tableCatalog) {
+    super(tableCatalog);
+  }
+
+  @Override
+  public ProcedureParameter[] parameters() {
+    return PARAMETERS;
+  }
+
+  @Override
+  public StructType outputType() {
+    return OUTPUT_TYPE;
+  }
+
+  @Override
+  public InternalRow[] call(InternalRow args) {
+    String tableName = args.getString(0);
+
+    // Read data from the table.changes
+    Dataset<Row> df = changelogRecords(tableName, args);
+
+    // compute updated rows and remove carry-over rows by default
+    boolean computeUpdatedRow = args.isNullAt(3) ? true : args.getBoolean(3);
+    boolean removeCarryoverRow = args.isNullAt(4) ? true : args.getBoolean(4);
+
+    if (computeUpdatedRow) {
+      String[] identifierColumns = identifierColumns(args, tableName);
+
+      if (identifierColumns.length > 0) {
+        Column[] repartitionColumns = getRepartitionExpr(df, identifierColumns);
+        df = transform(df, repartitionColumns);
+      } else {
+        LOG.warn("Cannot compute the update-rows because identifier columns are not set");
+        if (removeCarryoverRow) {
+          df = removeCarryoverRows(df);
+        }
+      }
+    } else if (removeCarryoverRow) {
+      df = removeCarryoverRows(df);
+    }
+
+    String viewName = viewName(args, tableName);
+
+    // Create a view for users to query
+    df.createOrReplaceTempView(viewName);
+
+    return toOutputRows(viewName);
+  }
+
+  private Dataset<Row> removeCarryoverRows(Dataset<Row> df) {
+    Column[] repartitionColumns =
+        Arrays.stream(df.columns())
+            .filter(c -> !c.equals(MetadataColumns.CHANGE_TYPE.name()))
+            .map(df::col)
+            .toArray(Column[]::new);
+    return transform(df, repartitionColumns);
+  }
+
+  private String[] identifierColumns(InternalRow args, String tableName) {
+    String[] identifierColumns = new String[0];
+    if (!args.isNullAt(5) && !args.getString(5).isEmpty()) {
+      identifierColumns = args.getString(5).split(",");
+    }
+
+    if (identifierColumns.length == 0) {
+      Identifier tableIdent = toIdentifier(tableName, PARAMETERS[0].name());
+      Table table = loadSparkTable(tableIdent).table();
+      identifierColumns = table.schema().identifierFieldNames().toArray(new String[0]);
+    }
+
+    return identifierColumns;
+  }
+
+  private Dataset<Row> changelogRecords(String tableName, InternalRow args) {
+    // no need to validate the read options here since the reader will validate them
+    DataFrameReader reader = spark().read();
+    reader.options(readOptions(args));
+    return reader.table(tableName + "." + SparkChangelogTable.TABLE_NAME);
+  }
+
+  private Map<String, String> readOptions(InternalRow args) {
+    Map<String, String> options = Maps.newHashMap();
+
+    if (!args.isNullAt(2)) {
+      args.getMap(2)
+          .foreach(
+              DataTypes.StringType,
+              DataTypes.StringType,
+              (k, v) -> {
+                if (k.toString().equals(SparkReadOptions.START_TIMESTAMP)
+                    || k.toString().equals(SparkReadOptions.END_TIMESTAMP)) {
+                  options.put(k.toString(), toMillis(v.toString()));

Review Comment:
   I'm a little lost on our conversion here, don't we already have code to convert this read option from String => millis within the reader itself?



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