unresolved operator 'ReplaceData when mergeInto involves not null join keys
See original GitHub issuespark 3.1.2 + iceberg 0.14.1 has been running successfully. spark 3.2.0 + iceberg 0.14.1 results in following error
Command
./bin/spark-shell --packages org.apache.iceberg:iceberg-spark-runtime-3.2_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.SparkSessionCatalog --conf spark.sql.catalog.spark_catalog.type=hive --conf spark.sql.catalog.local=org.apache.iceberg.spark.SparkCatalog --conf spark.sql.catalog.local.type=hadoop --conf spark.sql.catalog.local.warehouse=$PWD/warehouse;
Exception
org.apache.spark.sql.AnalysisException: unresolved operator 'ReplaceData RelationV2[id#219, firstname#220, lastname#221, age#222, date_id#223] local.db.target;
'MergeIntoIcebergTable (cast(id#219 as bigint) = id#224L), [deleteaction(Some(((operation_type#230 = DELETE) AND (isnull(date_id#223) OR (arrival_time#229 > date_id#223))))), updateaction(Some((((operation_type#230 = UPSERT) OR (operation_type#230 = APPEND)) AND (isnull(date_id#223) OR (arrival_time#229 > date_id#223)))), assignment(id#219, ansi_cast(id#224L as string)), assignment(firstname#220, firstname#225), assignment(lastname#221, lastname#226), assignment(age#222, age#227), assignment(date_id#223, date_id#228))], [insertaction(Some(NOT (operation_type#230 = DELETE)), assignment(id#219, ansi_cast(id#224L as string)), assignment(firstname#220, firstname#225), assignment(lastname#221, lastname#226), assignment(age#222, age#227), assignment(date_id#223, date_id#228))]
:- SubqueryAlias target
: +- SubqueryAlias local.db.target
: +- RelationV2[id#219, firstname#220, lastname#221, age#222, date_id#223] local.db.target
:- SubqueryAlias source
: +- SubqueryAlias local.db.source
: +- RelationV2[id#224L, firstname#225, lastname#226, age#227, date_id#228, arrival_time#229, operation_type#230] local.db.source
+- 'ReplaceData
+- MergeRows[id#219, firstname#220, lastname#221, age#222, date_id#223, _file#233]
+- Join FullOuter, (cast(id#219 as bigint) = id#244L), leftHint=(strategy=no_broadcast_hash)
:- NoStatsUnaryNode
: +- Project [id#219, firstname#220, lastname#221, age#222, date_id#223, _file#233, true AS __row_from_target#236, monotonically_increasing_id() AS __row_id#237L]
: +- RelationV2[id#219, firstname#220, lastname#221, age#222, date_id#223, _file#233] local.db.target
+- Project [id#244L, firstname#245, lastname#246, age#247, date_id#248, arrival_time#249, operation_type#250, true AS __row_from_source#238]
+- SubqueryAlias source
+- SubqueryAlias local.db.source
+- RelationV2[id#244L, firstname#245, lastname#246, age#247, date_id#248, arrival_time#249, operation_type#250] local.db.source
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis(CheckAnalysis.scala:51)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.failAnalysis$(CheckAnalysis.scala:50)
at org.apache.spark.sql.catalyst.analysis.Analyzer.failAnalysis(Analyzer.scala:172)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$38(CheckAnalysis.scala:553)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.$anonfun$checkAnalysis$38$adapted(CheckAnalysis.scala:551)
at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:263)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1(TreeNode.scala:262)
at org.apache.spark.sql.catalyst.trees.TreeNode.$anonfun$foreachUp$1$adapted(TreeNode.scala:262)
at scala.collection.immutable.List.foreach(List.scala:431)
at org.apache.spark.sql.catalyst.trees.TreeNode.foreachUp(TreeNode.scala:262)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis(CheckAnalysis.scala:551)
at org.apache.spark.sql.catalyst.analysis.CheckAnalysis.checkAnalysis$(CheckAnalysis.scala:91)
at org.apache.spark.sql.catalyst.analysis.Analyzer.checkAnalysis(Analyzer.scala:172)
at org.apache.spark.sql.catalyst.analysis.Analyzer.$anonfun$executeAndCheck$1(Analyzer.scala:195)
at org.apache.spark.sql.catalyst.plans.logical.AnalysisHelper$.markInAnalyzer(AnalysisHelper.scala:330)
at org.apache.spark.sql.catalyst.analysis.Analyzer.executeAndCheck(Analyzer.scala:192)
at org.apache.spark.sql.execution.QueryExecution.$anonfun$analyzed$1(QueryExecution.scala:88)
at org.apache.spark.sql.catalyst.QueryPlanningTracker.measurePhase(QueryPlanningTracker.scala:111)
at org.apache.spark.sql.execution.QueryExecution.$anonfun$executePhase$1(QueryExecution.scala:196)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.execution.QueryExecution.executePhase(QueryExecution.scala:196)
at org.apache.spark.sql.execution.QueryExecution.analyzed$lzycompute(QueryExecution.scala:88)
at org.apache.spark.sql.execution.QueryExecution.analyzed(QueryExecution.scala:86)
at org.apache.spark.sql.execution.QueryExecution.assertAnalyzed(QueryExecution.scala:78)
at org.apache.spark.sql.Dataset$.$anonfun$ofRows$2(Dataset.scala:98)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.Dataset$.ofRows(Dataset.scala:96)
at org.apache.spark.sql.SparkSession.$anonfun$sql$1(SparkSession.scala:618)
at org.apache.spark.sql.SparkSession.withActive(SparkSession.scala:775)
at org.apache.spark.sql.SparkSession.sql(SparkSession.scala:613)
... 58 elided
spark-shell code
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 local.db.target")
spark.sql("CREATE TABLE local.db.target (id string not null, firstname string, lastname string, age int, date_id timestamp) USING iceberg");
val namespace = "db"
val target = "target"
val targetName = TableIdentifier.of(namespace, target)
val catalog = new HadoopCatalog(spark.sparkContext.hadoopConfiguration, "file:///Users/mohit.garg/spark_iceberg/spark-3.2.0-bin-hadoop3.2/warehouse")
val targetTable = catalog.loadTable(targetName)
targetTable.updateProperties().set(TableProperties.FORMAT_VERSION, "2").commit();
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 local.db.target SELECT * from targetDf")
spark.sql("select * from local.db.target").show
// 2. Prepare source data
spark.sql("drop table local.db.source")
spark.sql("CREATE TABLE local.db.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 local.db.source SELECT * from sourceDf")
spark.sql("select * from local.db.source").show
// 3. MergeInto
spark.sql("""MERGE INTO local.db.target as target
USING local.db.source as 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 local.db.target").show
Issue Analytics
- State:
- Created a year ago
- Comments:5 (5 by maintainers)
Top Results From Across the Web
org.apache.spark.sql.AnalysisException: unresolved operator ...
I ran into the same situation and it turns out to be not only the fields need to be the same but also...
Read more >NULL Semantics - Spark 3.0.0-preview Documentation
A JOIN operator is used to combine rows from two tables based on a join condition. For all the three operators, a condition...
Read more >SELECT only IS NOT NULL columns with LEFT JOIN
Firstly - use INNER JOIN instead of LEFT (study the difference carefully !!!). Then, if some column in some options table may contain...
Read more >Inner join drops records in result - Azure Databricks
If you attempt to join tables, and some of the columns contain null values, the null records will not be included in the...
Read more >Joining on NULLs - Bert Wagner
Since it's not possible to join on NULL values in SQL Server like you ... a new column that will act as a...
Read more >Top Related Medium Post
No results found
Top Related StackOverflow Question
No results found
Troubleshoot Live Code
Lightrun enables developers to add logs, metrics and snapshots to live code - no restarts or redeploys required.
Start FreeTop Related Reddit Thread
No results found
Top Related Hackernoon Post
No results found
Top Related Tweet
No results found
Top Related Dev.to Post
No results found
Top Related Hashnode Post
No results found
Top GitHub Comments
Should be resolved by #5679 and #5880. Feel free to reopen if needed, @mohitgargk.
I can confirm this bug. Created a slightly modified script to reproduce this: