Fokko commented on issue #5739:
URL: https://github.com/apache/iceberg/issues/5739#issuecomment-1242903069

   I can confirm this bug. Created a slightly modified script to reproduce this:
   ```
   ./bin/spark-shell --packages 
org.apache.iceberg:iceberg-spark-runtime-3.1_2.12:0.14.1 \
   --conf 
spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions
 \
   --conf spark.sql.catalog.spark_catalog=org.apache.iceberg.spark.SparkCatalog 
\
   --conf spark.sql.catalog.spark_catalog.type=hadoop \
   --conf spark.sql.catalog.spark_catalog.warehouse=$PWD/warehouse
   ```
   
   ```scala
   import org.apache.spark.sql.Column;
   import org.apache.spark.sql.DataFrame;
   import org.apache.spark.sql.Row;
   import org.apache.spark.sql.types.StructField;
   import org.apache.spark.sql.types.StructType;
   import org.apache.iceberg.{PartitionSpec, Schema, Table}
   import org.apache.iceberg.catalog.TableIdentifier
   import org.apache.iceberg.hadoop.HadoopCatalog
   import org.apache.iceberg.types.Types
   import org.apache.iceberg.TableProperties
   import spark.implicits._
   import java.sql.Timestamp
   
   // Utility
   def setNullableStateOfColumn( df: DataFrame, cn: String, nullable: Boolean) 
: DataFrame = {
     val schema = df.schema
     val newSchema = StructType(schema.map {
       case StructField( c, t, _, m) if c.equals(cn) => StructField( c, t, 
nullable = nullable, m)
       case y: StructField => y
     })
     df.sqlContext.createDataFrame( df.rdd, newSchema )
   }
   
   // 1. Prepare target data
   spark.sql("DROP TABLE IF EXISTS target")
   spark.sql("CREATE TABLE target (id string not null, firstname string, 
lastname string, age int, date_id timestamp) USING iceberg");
   spark.sql("SELECT * FROM target").show
   
   val namespace = "default"
   val target = "target"
   val targetName = TableIdentifier.of(namespace, target)
   val catalog = new HadoopCatalog(spark.sparkContext.hadoopConfiguration, 
"/Users/fokkodriesprong/Desktop/spark-3.2.0-bin-hadoop3.2/warehouse")
   val targetTable = catalog.loadTable(targetName)
   targetTable.updateProperties().set(TableProperties.FORMAT_VERSION, 
"2").commit();
   
   // spark.sql("ALTER TABLE local.db.target SET('format-version'='2')").show()
   
   val ts = Timestamp.valueOf("2022-01-01 00:00:00");
   val range = (1 to 10).toList
   val targetData = range.map(id => (id.toString, "", "", 0, ts) )
   var targetDf = spark.createDataFrame(targetData).toDF("id", "firstname", 
"lastname", "age", "date_id")
   
   val newTargetDf = setNullableStateOfColumn(targetDf, "id", false)
   newTargetDf.registerTempTable("targetDf")
   
   spark.sql("INSERT INTO target SELECT * FROM targetDf")
   spark.sql("SELECT * FROM target").show
   
   // 2. Prepare source data
   spark.sql("DROP TABLE IF EXISTS source")
   spark.sql("CREATE TABLE source (id bigint not null, firstname string, 
lastname string, age int, date_id timestamp, arrival_time timestamp, 
operation_type string) USING iceberg");
   
   val source = "source"
   val sourceName = TableIdentifier.of(namespace, source)
   val sourceTable = catalog.loadTable(sourceName)
   sourceTable.updateProperties().set(TableProperties.FORMAT_VERSION, 
"2").commit();
   
   val sourceData = Seq( 
     (1, "mohit", "garg", 1, Timestamp.valueOf("2022-01-01 00:00:01"), 
Timestamp.valueOf("2022-01-01 00:00:01"), "UPSERT" ), 
     (2, "adam", "hancock", 1, Timestamp.valueOf("2022-01-01 00:00:01"), 
Timestamp.valueOf("2022-01-01 00:00:01"), "APPEND" ), 
     (3, "pradeep", "venkat", 1, Timestamp.valueOf("2022-01-01 00:00:01"), 
Timestamp.valueOf("2022-01-01 00:00:01"), "DELETE" ) ) ;
   var sourceDf = spark.createDataFrame(sourceData).toDF("id", "firstname", 
"lastname", "age", "date_id", "arrival_time", "operation_type")
   sourceDf.registerTempTable("sourceDf")
   val newSourceDf = setNullableStateOfColumn(sourceDf, "id", false)
   newSourceDf.registerTempTable("sourceDf")
   
   spark.sql("INSERT INTO source SELECT * FROM sourceDf")
   spark.sql("SELECT * FROM source").show
   
   // 3. MergeInto
   
   spark.sql("""MERGE INTO target
   USING source ON target.id = source.id 
     WHEN MATCHED AND source.operation_type = 'DELETE' AND (target.date_id IS 
NULL OR source.arrival_time > target.date_id) 
       THEN DELETE 
   
     WHEN MATCHED AND (source.operation_type = 'UPSERT' OR 
source.operation_type = 'APPEND') AND (target.date_id IS NULL OR 
source.arrival_time > target.date_id) 
       THEN UPDATE SET target.id = source.id, target.firstname = 
source.firstname, target.lastname = source.lastname, target.age = source.age, 
target.date_id = source.date_id 
   
     WHEN NOT MATCHED AND source.operation_type != 'DELETE' 
       THEN INSERT (`id`, `firstname`, `lastname`, `age`, `date_id`) VALUES 
(source.`id`, source.`firstname`, source.`lastname`, source.`age`, 
source.`date_id`)""")
   
   spark.sql("select * from target").show
   ```


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