diff --git a/spark-connect/client/src/main/scala-spark-master/io/delta/connect/tables/DeltaMergeBuilder.scala b/spark-connect/client/src/main/scala-spark-master/io/delta/connect/tables/DeltaMergeBuilder.scala index a84ec01bba..d4f2de9cf1 100644 --- a/spark-connect/client/src/main/scala-spark-master/io/delta/connect/tables/DeltaMergeBuilder.scala +++ b/spark-connect/client/src/main/scala-spark-master/io/delta/connect/tables/DeltaMergeBuilder.scala @@ -488,6 +488,13 @@ class DeltaMergeMatchedActionBuilder private( } } + /** + * Converts a map of strings to expressions as SQL formatted string + * into a map of strings to Column objects. + * + * @param map A map where the value is an expression as SQL formatted string. + * @return A map where the value is a Column object created from the expression. + */ private def toStrColumnMap(map: Map[String, String]): Map[String, Column] = map.mapValues(functions.expr).toMap } diff --git a/spark-connect/server/src/test/scala-spark-master/io/delta/connect/DeltaConnectPlannerSuite.scala b/spark-connect/server/src/test/scala-spark-master/io/delta/connect/DeltaConnectPlannerSuite.scala index 5fc6eb5958..e6cf16b6b3 100644 --- a/spark-connect/server/src/test/scala-spark-master/io/delta/connect/DeltaConnectPlannerSuite.scala +++ b/spark-connect/server/src/test/scala-spark-master/io/delta/connect/DeltaConnectPlannerSuite.scala @@ -485,7 +485,6 @@ class DeltaConnectPlannerSuite ) val plan = transform(input) - assert(plan.columns.toSeq === V2CommandOutputs.mergeOutput.map(_.name)) val result = Dataset.ofRows(spark, plan).collect() assert(result.length === 1) assert(result.head.getLong(0) === 50) // num_affected_rows @@ -531,7 +530,6 @@ class DeltaConnectPlannerSuite ) val plan = transform(input) - assert(plan.columns.toSeq === V2CommandOutputs.mergeOutput.map(_.name)) val result = Dataset.ofRows(spark, plan).collect() assert(result.length === 1) assert(result.head.getLong(0) === 50) // num_affected_rows @@ -582,7 +580,6 @@ class DeltaConnectPlannerSuite ) val plan = transform(input) - assert(plan.columns.toSeq === V2CommandOutputs.mergeOutput.map(_.name)) val result = Dataset.ofRows(spark, plan).collect() assert(result.length === 1) assert(result.head.getLong(0) === 125) // num_affected_rows @@ -635,9 +632,7 @@ class DeltaConnectPlannerSuite ) ) - val plan = new SparkConnectPlanner( - SparkConnectTestUtils.createDummySessionHolder(spark)).transformRelation(input) - assert(plan.columns.toSeq === V2CommandOutputs.mergeOutput.map(_.name)) + val plan = transform(input) val result = Dataset.ofRows(spark, plan).collect() assert(result.length === 1) assert(result.head.getLong(0) === 125) // num_affected_rows @@ -690,9 +685,7 @@ class DeltaConnectPlannerSuite ) ) - val plan = new SparkConnectPlanner( - SparkConnectTestUtils.createDummySessionHolder(spark)).transformRelation(input) - assert(plan.columns.toSeq === V2CommandOutputs.mergeOutput.map(_.name)) + val plan = transform(input) val result = Dataset.ofRows(spark, plan).collect() assert(result.length === 1) assert(result.head.getLong(0) === 125) // num_affected_rows