rdblue commented on code in PR #5094:
URL: https://github.com/apache/iceberg/pull/5094#discussion_r902010881


##########
spark/v3.3/spark-extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala:
##########
@@ -0,0 +1,387 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.spark.sql.catalyst.analysis
+
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.expressions.Alias
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.expressions.AttributeReference
+import org.apache.spark.sql.catalyst.expressions.AttributeSet
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.expressions.ExtendedV2ExpressionUtils
+import org.apache.spark.sql.catalyst.expressions.IsNotNull
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral
+import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral
+import org.apache.spark.sql.catalyst.expressions.MonotonicallyIncreasingID
+import org.apache.spark.sql.catalyst.plans.FullOuter
+import org.apache.spark.sql.catalyst.plans.Inner
+import org.apache.spark.sql.catalyst.plans.LeftAnti
+import org.apache.spark.sql.catalyst.plans.LeftOuter
+import org.apache.spark.sql.catalyst.plans.RightOuter
+import org.apache.spark.sql.catalyst.plans.logical.AppendData
+import org.apache.spark.sql.catalyst.plans.logical.DeleteAction
+import org.apache.spark.sql.catalyst.plans.logical.Filter
+import org.apache.spark.sql.catalyst.plans.logical.HintInfo
+import org.apache.spark.sql.catalyst.plans.logical.InsertAction
+import org.apache.spark.sql.catalyst.plans.logical.Join
+import org.apache.spark.sql.catalyst.plans.logical.JoinHint
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.plans.logical.MergeAction
+import org.apache.spark.sql.catalyst.plans.logical.MergeIntoIcebergTable
+import org.apache.spark.sql.catalyst.plans.logical.MergeRows
+import org.apache.spark.sql.catalyst.plans.logical.NO_BROADCAST_HASH
+import org.apache.spark.sql.catalyst.plans.logical.NoStatsUnaryNode
+import org.apache.spark.sql.catalyst.plans.logical.Project
+import org.apache.spark.sql.catalyst.plans.logical.ReplaceIcebergData
+import org.apache.spark.sql.catalyst.plans.logical.UpdateAction
+import org.apache.spark.sql.catalyst.plans.logical.WriteDelta
+import org.apache.spark.sql.catalyst.util.RowDeltaUtils._
+import org.apache.spark.sql.connector.catalog.SupportsRowLevelOperations
+import org.apache.spark.sql.connector.expressions.FieldReference
+import org.apache.spark.sql.connector.expressions.NamedReference
+import org.apache.spark.sql.connector.iceberg.write.SupportsDelta
+import org.apache.spark.sql.connector.write.RowLevelOperation.Command.MERGE
+import org.apache.spark.sql.connector.write.RowLevelOperationTable
+import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation
+import org.apache.spark.sql.types.IntegerType
+import org.apache.spark.sql.util.CaseInsensitiveStringMap
+
+/**
+ * Assigns a rewrite plan for v2 tables that support rewriting data to handle 
MERGE statements.
+ *
+ * This rule assumes the commands have been fully resolved and all assignments 
have been aligned.
+ * That's why it must be run after AlignRowLevelCommandAssignments.
+ */
+object RewriteMergeIntoTable extends RewriteRowLevelDeltaCommand with 
RewriteRowLevelCommand {
+
+  private final val ROW_FROM_SOURCE = "__row_from_source"
+  private final val ROW_FROM_TARGET = "__row_from_target"
+  private final val ROW_ID = "__row_id"
+
+  private final val ROW_FROM_SOURCE_REF = FieldReference(ROW_FROM_SOURCE)
+  private final val ROW_FROM_TARGET_REF = FieldReference(ROW_FROM_TARGET)
+  private final val ROW_ID_REF = FieldReference(ROW_ID)
+
+  override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned && matchedActions.isEmpty && 
notMatchedActions.size == 1 =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r: DataSourceV2Relation =>
+          // NOT MATCHED conditions may only refer to columns in source so 
they can be pushed down
+          val insertAction = notMatchedActions.head.asInstanceOf[InsertAction]
+          val filteredSource = insertAction.condition match {
+            case Some(insertCond) => Filter(insertCond, source)
+            case None => source
+          }
+
+          // when there are no MATCHED actions, use a left anti join to remove 
any matching rows
+          // and switch to using a regular append instead of a row-level merge
+          // only unmatched source rows that match the condition are appended 
to the table
+          val joinPlan = Join(filteredSource, r, LeftAnti, Some(cond), 
JoinHint.NONE)
+
+          val outputExprs = insertAction.assignments.map(_.value)
+          val outputColNames = r.output.map(_.name)
+          val outputCols = outputExprs.zip(outputColNames).map { case (expr, 
name) =>
+            Alias(expr, name)()
+          }
+          val project = Project(outputCols, joinPlan)
+
+          AppendData.byPosition(r, project)
+
+        case p =>
+          throw new AnalysisException(s"$p is not an Iceberg table")
+      }
+
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned && matchedActions.isEmpty =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r: DataSourceV2Relation =>
+          // when there are no MATCHED actions, use a left anti join to remove 
any matching rows
+          // and switch to using a regular append instead of a row-level merge
+          // only unmatched source rows that match action conditions are 
appended to the table
+          val joinPlan = Join(source, r, LeftAnti, Some(cond), JoinHint.NONE)
+
+          val notMatchedConditions = notMatchedActions.map(actionCondition)
+          val notMatchedOutputs = notMatchedActions.map(actionOutput(_, Nil))
+
+          // merge rows as there are multiple not matched actions
+          val mergeRows = MergeRows(
+            isSourceRowPresent = TrueLiteral,
+            isTargetRowPresent = FalseLiteral,
+            matchedConditions = Nil,
+            matchedOutputs = Nil,
+            notMatchedConditions = notMatchedConditions,
+            notMatchedOutputs = notMatchedOutputs,
+            targetOutput = Nil,
+            rowIdAttrs = Nil,
+            performCardinalityCheck = false,
+            emitNotMatchedTargetRows = false,
+            output = buildMergeRowsOutput(Nil, notMatchedOutputs, r.output),
+            joinPlan)
+
+          AppendData.byPosition(r, mergeRows)
+
+        case p =>
+          throw new AnalysisException(s"$p is not an Iceberg table")
+      }
+
+    case m @ MergeIntoIcebergTable(aliasedTable, source, cond, matchedActions, 
notMatchedActions, None)
+        if m.resolved && m.aligned =>
+
+      EliminateSubqueryAliases(aliasedTable) match {
+        case r @ DataSourceV2Relation(tbl: SupportsRowLevelOperations, _, _, 
_, _) =>
+          val table = buildOperationTable(tbl, MERGE, 
CaseInsensitiveStringMap.empty())

Review Comment:
   Why did this change? Are you using utilities from Spark now?



##########
spark/v3.3/spark/src/test/java/org/apache/iceberg/spark/actions/TestCreateActions.java:
##########
@@ -0,0 +1,898 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.actions;
+
+import java.io.File;
+import java.io.FilenameFilter;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.file.Paths;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.filefilter.TrueFileFilter;
+import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.Schema;
+import org.apache.iceberg.Table;
+import org.apache.iceberg.actions.MigrateTable;
+import org.apache.iceberg.actions.SnapshotTable;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
+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.Spark3Util;
+import org.apache.iceberg.spark.SparkCatalog;
+import org.apache.iceberg.spark.SparkCatalogTestBase;
+import org.apache.iceberg.spark.SparkSessionCatalog;
+import org.apache.iceberg.spark.source.SimpleRecord;
+import org.apache.iceberg.spark.source.SparkTable;
+import org.apache.iceberg.types.Types;
+import org.apache.parquet.hadoop.ParquetFileReader;
+import org.apache.parquet.hadoop.util.HadoopInputFile;
+import org.apache.parquet.schema.MessageType;
+import org.apache.parquet.schema.MessageTypeParser;
+import org.apache.spark.api.java.JavaSparkContext;
+import org.apache.spark.sql.Dataset;
+import org.apache.spark.sql.Row;
+import org.apache.spark.sql.SaveMode;
+import org.apache.spark.sql.catalyst.TableIdentifier;
+import org.apache.spark.sql.catalyst.analysis.NoSuchDatabaseException;
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException;
+import org.apache.spark.sql.catalyst.catalog.CatalogTable;
+import org.apache.spark.sql.catalyst.catalog.CatalogTablePartition;
+import org.apache.spark.sql.catalyst.parser.ParseException;
+import org.apache.spark.sql.connector.catalog.Identifier;
+import org.apache.spark.sql.connector.catalog.TableCatalog;
+import org.apache.spark.sql.types.ArrayType;
+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.junit.After;
+import org.junit.Assert;
+import org.junit.Assume;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+import org.junit.runners.Parameterized;
+import scala.Option;
+import scala.Some;
+import scala.collection.JavaConverters;
+import scala.collection.Seq;
+
+public class TestCreateActions extends SparkCatalogTestBase {
+  private static final String CREATE_PARTITIONED_PARQUET = "CREATE TABLE %s 
(id INT, data STRING) " +
+      "using parquet PARTITIONED BY (id) LOCATION '%s'";
+  private static final String CREATE_PARQUET = "CREATE TABLE %s (id INT, data 
STRING) " +
+      "using parquet LOCATION '%s'";
+  private static final String CREATE_HIVE_EXTERNAL_PARQUET = "CREATE EXTERNAL 
TABLE %s (data STRING) " +
+      "PARTITIONED BY (id INT) STORED AS parquet LOCATION '%s'";
+  private static final String CREATE_HIVE_PARQUET = "CREATE TABLE %s (data 
STRING) " +
+      "PARTITIONED BY (id INT) STORED AS parquet";
+
+  private static final String NAMESPACE = "default";
+
+  @Parameterized.Parameters(name = "Catalog Name {0} - Options {2}")
+  public static Object[][] parameters() {
+    return new Object[][] {
+        new Object[] {"spark_catalog", SparkSessionCatalog.class.getName(), 
ImmutableMap.of(
+            "type", "hive",
+            "default-namespace", "default",
+            "parquet-enabled", "true",
+            "cache-enabled", "false" // Spark will delete tables using v1, 
leaving the cache out of sync
+        )},
+        new Object[] {"spark_catalog", SparkSessionCatalog.class.getName(), 
ImmutableMap.of(
+            "type", "hadoop",
+            "default-namespace", "default",
+            "parquet-enabled", "true",
+            "cache-enabled", "false" // Spark will delete tables using v1, 
leaving the cache out of sync
+        )},
+        new Object[] { "testhive", SparkCatalog.class.getName(), 
ImmutableMap.of(
+            "type", "hive",
+            "default-namespace", "default"
+        )},
+        new Object[] { "testhadoop", SparkCatalog.class.getName(), 
ImmutableMap.of(
+            "type", "hadoop",
+            "default-namespace", "default"
+        )}
+    };
+  }
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  private String baseTableName = "baseTable";
+  private File tableDir;
+  private String tableLocation;
+  private final String type;
+  private final TableCatalog catalog;
+
+  public TestCreateActions(
+      String catalogName,
+      String implementation,
+      Map<String, String> config) {
+    super(catalogName, implementation, config);
+    this.catalog = (TableCatalog) 
spark.sessionState().catalogManager().catalog(catalogName);
+    this.type = config.get("type");
+  }
+
+  @Before
+  public void before() {
+    try {
+      this.tableDir = temp.newFolder();
+    } catch (IOException e) {
+      throw new RuntimeException(e);
+    }
+    this.tableLocation = tableDir.toURI().toString();
+
+    spark.conf().set("hive.exec.dynamic.partition", "true");
+    spark.conf().set("hive.exec.dynamic.partition.mode", "nonstrict");
+    spark.conf().set("spark.sql.parquet.writeLegacyFormat", false);
+    spark.conf().set("spark.sql.parquet.writeLegacyFormat", false);
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName));
+
+    List<SimpleRecord> expected = Lists.newArrayList(
+        new SimpleRecord(1, "a"),
+        new SimpleRecord(2, "b"),
+        new SimpleRecord(3, "c")
+    );
+
+    Dataset<Row> df = spark.createDataFrame(expected, SimpleRecord.class);
+
+    df.select("id", "data").orderBy("data").write()
+        .mode("append")
+        .option("path", tableLocation)
+        .saveAsTable(baseTableName);
+  }
+
+  @After
+  public void after() throws IOException {
+    // Drop the hive table.
+    spark.sql(String.format("DROP TABLE IF EXISTS %s", baseTableName));
+  }
+
+  @Test
+  public void testMigratePartitioned() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_migrate_partitioned_table");
+    String dest = source;
+    createSourceTable(CREATE_PARTITIONED_PARQUET, source);
+    assertMigratedFileCount(SparkActions.get().migrateTable(source), source, 
dest);
+  }
+
+  @Test
+  public void testPartitionedTableWithUnRecoveredPartitions() throws Exception 
{
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_unrecovered_partitions");
+    String dest = source;
+    File location = temp.newFolder();
+    sql(CREATE_PARTITIONED_PARQUET, source, location);
+
+    // Data generation and partition addition
+    spark.range(5)
+        .selectExpr("id", "cast(id as STRING) as data")
+        .write()
+        .partitionBy("id").mode(SaveMode.Overwrite)
+        .parquet(location.toURI().toString());
+    sql("ALTER TABLE %s ADD PARTITION(id=0)", source);
+
+    assertMigratedFileCount(SparkActions.get().migrateTable(source), source, 
dest);
+  }
+
+  @Test
+  public void testPartitionedTableWithCustomPartitions() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_custom_parts");
+    String dest = source;
+    File tblLocation = temp.newFolder();
+    File partitionDataLoc = temp.newFolder();
+
+    // Data generation and partition addition
+    spark.sql(String.format(CREATE_PARTITIONED_PARQUET, source, tblLocation));
+    spark.range(10)
+        .selectExpr("cast(id as STRING) as data")
+        .write()
+        .mode(SaveMode.Overwrite).parquet(partitionDataLoc.toURI().toString());
+    sql("ALTER TABLE %s ADD PARTITION(id=0) LOCATION '%s'", source, 
partitionDataLoc.toURI().toString());
+    assertMigratedFileCount(SparkActions.get().migrateTable(source), source, 
dest);
+  }
+
+  @Test
+  public void testAddColumnOnMigratedTableAtEnd() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_add_column_migrated_table");
+    String dest = source;
+    createSourceTable(CREATE_PARQUET, source);
+    List<Object[]> expected1 = sql("select *, null from %s order by id", 
source);
+    List<Object[]> expected2 = sql("select *, null, null from %s order by id", 
source);
+
+    // migrate table
+    SparkActions.get().migrateTable(source).execute();
+    SparkTable sparkTable = loadTable(dest);
+    Table table = sparkTable.table();
+
+    // test column addition on migrated table
+    Schema beforeSchema = table.schema();
+    String newCol1 = "newCol1";
+    sparkTable.table().updateSchema().addColumn(newCol1, 
Types.IntegerType.get()).commit();
+    Schema afterSchema = table.schema();
+    Assert.assertNull(beforeSchema.findField(newCol1));
+    Assert.assertNotNull(afterSchema.findField(newCol1));
+
+    // reads should succeed without any exceptions
+    List<Object[]> results1 = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results1.size() > 0);
+    assertEquals("Output must match", results1, expected1);
+
+    String newCol2 = "newCol2";
+    sql("ALTER TABLE %s ADD COLUMN %s INT", dest, newCol2);
+    StructType schema = spark.table(dest).schema();
+    Assert.assertTrue(Arrays.asList(schema.fieldNames()).contains(newCol2));
+
+    // reads should succeed without any exceptions
+    List<Object[]> results2 = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results2.size() > 0);
+    assertEquals("Output must match", results2, expected2);
+  }
+
+  @Test
+  public void testAddColumnOnMigratedTableAtMiddle() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_add_column_migrated_table_middle");
+    String dest = source;
+    createSourceTable(CREATE_PARQUET, source);
+
+    // migrate table
+    SparkActions.get().migrateTable(source).execute();
+    SparkTable sparkTable = loadTable(dest);
+    Table table = sparkTable.table();
+    List<Object[]> expected = sql("select id, null, data from %s order by id", 
source);
+
+    // test column addition on migrated table
+    Schema beforeSchema = table.schema();
+    String newCol1 = "newCol";
+    sparkTable.table().updateSchema().addColumn("newCol", 
Types.IntegerType.get())
+        .moveAfter(newCol1, "id")
+        .commit();
+    Schema afterSchema = table.schema();
+    Assert.assertNull(beforeSchema.findField(newCol1));
+    Assert.assertNotNull(afterSchema.findField(newCol1));
+
+    // reads should succeed
+    List<Object[]> results = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results.size() > 0);
+    assertEquals("Output must match", results, expected);
+  }
+
+  @Test
+  public void removeColumnsAtEnd() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_remove_column_migrated_table");
+    String dest = source;
+
+    String colName1 = "newCol1";
+    String colName2 = "newCol2";
+    File location = temp.newFolder();
+    spark.range(10).selectExpr("cast(id as INT)", "CAST(id as INT) " + 
colName1, "CAST(id as INT) " + colName2)
+        .write()
+        .mode(SaveMode.Overwrite).saveAsTable(dest);
+    List<Object[]> expected1 = sql("select id, %s from %s order by id", 
colName1, source);
+    List<Object[]> expected2 = sql("select id from %s order by id", source);
+
+    // migrate table
+    SparkActions.get().migrateTable(source).execute();
+    SparkTable sparkTable = loadTable(dest);
+    Table table = sparkTable.table();
+
+    // test column removal on migrated table
+    Schema beforeSchema = table.schema();
+    sparkTable.table().updateSchema().deleteColumn(colName1).commit();
+    Schema afterSchema = table.schema();
+    Assert.assertNotNull(beforeSchema.findField(colName1));
+    Assert.assertNull(afterSchema.findField(colName1));
+
+    // reads should succeed without any exceptions
+    List<Object[]> results1 = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results1.size() > 0);
+    assertEquals("Output must match", expected1, results1);
+
+    sql("ALTER TABLE %s DROP COLUMN %s", dest, colName2);
+    StructType schema = spark.table(dest).schema();
+    Assert.assertFalse(Arrays.asList(schema.fieldNames()).contains(colName2));
+
+    // reads should succeed without any exceptions
+    List<Object[]> results2 = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results2.size() > 0);
+    assertEquals("Output must match", expected2, results2);
+  }
+
+  @Test
+  public void removeColumnFromMiddle() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = 
sourceName("test_remove_column_migrated_table_from_middle");
+    String dest = source;
+    String dropColumnName = "col1";
+
+    spark.range(10).selectExpr("cast(id as INT)", "CAST(id as INT) as " +
+        dropColumnName, "CAST(id as INT) as 
col2").write().mode(SaveMode.Overwrite).saveAsTable(dest);
+    List<Object[]> expected = sql("select id, col2 from %s order by id", 
source);
+
+    // migrate table
+    SparkActions.get().migrateTable(source).execute();
+
+    // drop column
+    sql("ALTER TABLE %s DROP COLUMN %s", dest, "col1");
+    StructType schema = spark.table(dest).schema();
+    
Assert.assertFalse(Arrays.asList(schema.fieldNames()).contains(dropColumnName));
+
+    // reads should return same output as that of non-iceberg table
+    List<Object[]> results = sql("select * from %s order by id", dest);
+    Assert.assertTrue(results.size() > 0);
+    assertEquals("Output must match", expected, results);
+  }
+
+  @Test
+  public void testMigrateUnpartitioned() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String source = sourceName("test_migrate_unpartitioned_table");
+    String dest = source;
+    createSourceTable(CREATE_PARQUET, source);
+    assertMigratedFileCount(SparkActions.get().migrateTable(source), source, 
dest);
+  }
+
+  @Test
+  public void testSnapshotPartitioned() throws Exception {
+    Assume.assumeTrue("Cannot snapshot with arbitrary location in a hadoop 
based catalog",
+        !type.equals("hadoop"));
+    File location = temp.newFolder();
+    String source = sourceName("test_snapshot_partitioned_table");
+    String dest = destName("iceberg_snapshot_partitioned");
+    createSourceTable(CREATE_PARTITIONED_PARQUET, source);
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()),
 source, dest);
+    assertIsolatedSnapshot(source, dest);
+  }
+
+  @Test
+  public void testSnapshotUnpartitioned() throws Exception {
+    Assume.assumeTrue("Cannot snapshot with arbitrary location in a hadoop 
based catalog",
+        !type.equals("hadoop"));
+    File location = temp.newFolder();
+    String source = sourceName("test_snapshot_unpartitioned_table");
+    String dest = destName("iceberg_snapshot_unpartitioned");
+    createSourceTable(CREATE_PARQUET, source);
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()),
 source, dest);
+    assertIsolatedSnapshot(source, dest);
+  }
+
+  @Test
+  public void testSnapshotHiveTable() throws Exception {
+    Assume.assumeTrue("Cannot snapshot with arbitrary location in a hadoop 
based catalog",
+        !type.equals("hadoop"));
+    File location = temp.newFolder();
+    String source = sourceName("snapshot_hive_table");
+    String dest = destName("iceberg_snapshot_hive_table");
+    createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source);
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()),
 source, dest);
+    assertIsolatedSnapshot(source, dest);
+  }
+
+  @Test
+  public void testMigrateHiveTable() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    String source = sourceName("migrate_hive_table");
+    String dest = source;
+    createSourceTable(CREATE_HIVE_EXTERNAL_PARQUET, source);
+    assertMigratedFileCount(SparkActions.get().migrateTable(source), source, 
dest);
+  }
+
+  @Test
+  public void testSnapshotManagedHiveTable() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    File location = temp.newFolder();
+    String source = sourceName("snapshot_managed_hive_table");
+    String dest = destName("iceberg_snapshot_managed_hive_table");
+    createSourceTable(CREATE_HIVE_PARQUET, source);
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()),
 source, dest);
+    assertIsolatedSnapshot(source, dest);
+  }
+
+  @Test
+  public void testMigrateManagedHiveTable() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    File location = temp.newFolder();
+    String source = sourceName("migrate_managed_hive_table");
+    String dest = destName("iceberg_migrate_managed_hive_table");
+    createSourceTable(CREATE_HIVE_PARQUET, source);
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableLocation(location.toString()),
 source, dest);
+  }
+
+  @Test
+  public void testProperties() throws Exception {
+    String source = sourceName("test_properties_table");
+    String dest = destName("iceberg_properties");
+    Map<String, String> props = Maps.newHashMap();
+    props.put("city", "New Orleans");
+    props.put("note", "Jazz");
+    createSourceTable(CREATE_PARQUET, source);
+    for (Map.Entry<String, String> keyValue : props.entrySet()) {
+      spark.sql(String.format("ALTER TABLE %s SET TBLPROPERTIES (\"%s\" = 
\"%s\")",
+          source, keyValue.getKey(), keyValue.getValue()));
+    }
+    assertSnapshotFileCount(
+        
SparkActions.get().snapshotTable(source).as(dest).tableProperty("dogs", 
"sundance"), source, dest);
+    SparkTable table = loadTable(dest);
+
+    Map<String, String> expectedProps = Maps.newHashMap();
+    expectedProps.putAll(props);
+    expectedProps.put("dogs", "sundance");
+
+    for (Map.Entry<String, String> entry : expectedProps.entrySet()) {
+      Assert.assertTrue(
+          "Created table missing property " + entry.getKey(),
+          table.properties().containsKey(entry.getKey()));
+      Assert.assertEquals("Property value is not the expected value",
+          entry.getValue(), table.properties().get(entry.getKey()));
+    }
+  }
+
+  @Test
+  public void testSparkTableReservedProperties() throws Exception {
+    String destTableName = "iceberg_reserved_properties";
+    String source = sourceName("test_reserved_properties_table");
+    String dest = destName(destTableName);
+    createSourceTable(CREATE_PARQUET, source);
+    assertSnapshotFileCount(SparkActions.get().snapshotTable(source).as(dest), 
source, dest);
+    SparkTable table = loadTable(dest);
+    // set sort orders
+    table.table().replaceSortOrder().asc("id").desc("data").commit();
+
+    String[] keys = {"provider", "format", "current-snapshot-id", "location", 
"sort-order"};
+
+    for (String entry : keys) {
+      Assert.assertTrue("Created table missing reserved property " + entry, 
table.properties().containsKey(entry));
+    }
+
+    Assert.assertEquals("Unexpected provider", "iceberg", 
table.properties().get("provider"));
+    Assert.assertEquals("Unexpected format", "iceberg/parquet", 
table.properties().get("format"));
+    Assert.assertNotEquals("No current-snapshot-id found", "none", 
table.properties().get("current-snapshot-id"));
+    Assert.assertTrue("Location isn't correct", 
table.properties().get("location").endsWith(destTableName));
+
+    Assert.assertEquals("Unexpected format-version", "1", 
table.properties().get("format-version"));
+    table.table().updateProperties().set("format-version", "2").commit();
+    Assert.assertEquals("Unexpected format-version", "2", 
table.properties().get("format-version"));
+
+    Assert.assertEquals("Sort-order isn't correct", "id ASC NULLS FIRST, data 
DESC NULLS LAST",
+        table.properties().get("sort-order"));
+    Assert.assertNull("Identifier fields should be null", 
table.properties().get("identifier-fields"));
+
+    
table.table().updateSchema().allowIncompatibleChanges().requireColumn("id").setIdentifierFields("id").commit();
+    Assert.assertEquals("Identifier fields aren't correct", "[id]", 
table.properties().get("identifier-fields"));
+  }
+
+  @Test
+  public void testSnapshotDefaultLocation() throws Exception {
+    String source = sourceName("test_snapshot_default");
+    String dest = destName("iceberg_snapshot_default");
+    createSourceTable(CREATE_PARTITIONED_PARQUET, source);
+    assertSnapshotFileCount(SparkActions.get().snapshotTable(source).as(dest), 
source, dest);
+    assertIsolatedSnapshot(source, dest);
+  }
+
+  @Test
+  public void schemaEvolutionTestWithSparkAPI() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+
+    File location = temp.newFolder();
+    String tblName = sourceName("schema_evolution_test");
+
+    // Data generation and partition addition
+    spark.range(0, 5)
+        .selectExpr("CAST(id as INT) as col0", "CAST(id AS FLOAT) col2", 
"CAST(id AS LONG) col3")
+        .write()
+        .mode(SaveMode.Append)
+        .parquet(location.toURI().toString());
+    Dataset<Row> rowDataset = spark.range(6, 10)
+        .selectExpr("CAST(id as INT) as col0", "CAST(id AS STRING) col1",
+            "CAST(id AS FLOAT) col2", "CAST(id AS LONG) col3");
+    rowDataset
+        .write()
+        .mode(SaveMode.Append)
+        .parquet(location.toURI().toString());
+    spark.read()
+        .schema(rowDataset.schema())
+        .parquet(location.toURI().toString()).write().saveAsTable(tblName);
+    List<Object[]> expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY 
col0", tblName);
+    List<Object[]> expectedAfterAddColumn = sql("SELECT col0, null, col1, 
col2, col3 FROM %s ORDER BY col0",
+        tblName);
+
+    // Migrate table
+    SparkActions.get().migrateTable(tblName).execute();
+
+    // check if iceberg and non-iceberg output
+    List<Object[]> afterMigarteBeforeAddResults = sql("SELECT * FROM %s ORDER 
BY col0", tblName);
+    assertEquals("Output must match", expectedBeforeAddColumn, 
afterMigarteBeforeAddResults);
+
+    // Update schema and check output correctness
+    SparkTable sparkTable = loadTable(tblName);
+    sparkTable.table().updateSchema().addColumn("newCol", 
Types.IntegerType.get())
+        .moveAfter("newCol", "col0")
+        .commit();
+    List<Object[]> afterMigarteAfterAddResults = sql("SELECT * FROM %s ORDER 
BY col0", tblName);
+    assertEquals("Output must match", expectedAfterAddColumn, 
afterMigarteAfterAddResults);
+  }
+
+  @Test
+  public void schemaEvolutionTestWithSparkSQL() throws Exception {
+    Assume.assumeTrue("Cannot migrate to a hadoop based catalog", 
!type.equals("hadoop"));
+    Assume.assumeTrue("Can only migrate from Spark Session Catalog", 
catalog.name().equals("spark_catalog"));
+    String tblName = sourceName("schema_evolution_test_sql");
+
+    // Data generation and partition addition
+    spark.range(0, 5)
+        .selectExpr("CAST(id as INT) col0", "CAST(id AS FLOAT) col1", "CAST(id 
AS STRING) col2")
+        .write()
+        .mode(SaveMode.Append)
+        .saveAsTable(tblName);
+    sql("ALTER TABLE %s ADD COLUMN col3 INT", tblName);
+    spark.range(6, 10)
+        .selectExpr("CAST(id AS INT) col0", "CAST(id AS FLOAT) col1", "CAST(id 
AS STRING) col2", "CAST(id AS INT) col3")
+        .registerTempTable("tempdata");
+    sql("INSERT INTO TABLE %s SELECT * FROM tempdata", tblName);
+    List<Object[]> expectedBeforeAddColumn = sql("SELECT * FROM %s ORDER BY 
col0", tblName);
+    List<Object[]> expectedAfterAddColumn = sql("SELECT col0, null, col1, 
col2, col3 FROM %s ORDER BY col0",
+        tblName);
+
+    // Migrate table
+    SparkActions.get().migrateTable(tblName).execute();
+
+    // check if iceberg and non-iceberg output
+    List<Object[]> afterMigarteBeforeAddResults = sql("SELECT * FROM %s ORDER 
BY col0", tblName);
+    assertEquals("Output must match", expectedBeforeAddColumn, 
afterMigarteBeforeAddResults);
+
+    // Update schema and check output correctness
+    SparkTable sparkTable = loadTable(tblName);
+    sparkTable.table().updateSchema().addColumn("newCol", 
Types.IntegerType.get())
+        .moveAfter("newCol", "col0")
+        .commit();
+    List<Object[]> afterMigarteAfterAddResults = sql("SELECT * FROM %s ORDER 
BY col0", tblName);
+    assertEquals("Output must match", expectedAfterAddColumn, 
afterMigarteAfterAddResults);
+  }
+
+  @Test
+  public void testHiveStyleThreeLevelList() throws Exception {
+    threeLevelList(true);
+  }
+
+  @Test
+  public void testThreeLevelList() throws Exception {
+    threeLevelList(false);
+  }
+
+  @Test
+  public void testHiveStyleThreeLevelListWithNestedStruct() throws Exception {
+    threeLevelListWithNestedStruct(true);
+  }
+
+  @Test
+  public void testThreeLevelListWithNestedStruct() throws Exception {
+    threeLevelListWithNestedStruct(false);
+  }
+
+  @Test
+  public void testHiveStyleThreeLevelLists() throws Exception {
+    threeLevelLists(true);
+  }
+
+  @Test
+  public void testThreeLevelLists() throws Exception {
+    threeLevelLists(false);
+  }
+
+  @Test
+  public void testHiveStyleStructOfThreeLevelLists() throws Exception {
+    structOfThreeLevelLists(true);
+  }
+
+  @Test
+  public void testStructOfThreeLevelLists() throws Exception {
+    structOfThreeLevelLists(false);
+  }
+
+  @Test
+  public void testTwoLevelList() throws IOException {
+    spark.conf().set("spark.sql.parquet.writeLegacyFormat", true);
+
+    String tableName = sourceName("testTwoLevelList");
+    File location = temp.newFolder();
+
+    StructType sparkSchema =
+        new StructType(
+            new StructField[]{
+                new StructField(
+                        "col1", new ArrayType(
+                            new StructType(
+                                new StructField[]{
+                                    new StructField(
+                                        "col2",
+                                        DataTypes.IntegerType,
+                                        false,
+                                        Metadata.empty())
+                                }), false), true, Metadata.empty())});
+
+    // even though this list looks like three level list, it is actually a 
2-level list where the items are
+    // structs with 1 field.
+    String expectedParquetSchema =
+        "message spark_schema {\n" +
+            "  optional group col1 (LIST) {\n" +
+            "    repeated group bag {\n" +

Review Comment:
   This change isn't correct. This was validating that Spark actually wrote a 
two-level list. But this is now validating a 3-level list. We'll need to find 
out why Spark is no longer writing the legacy format. Can you change this back 
and set the test to `@Ignore` instead?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to