You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "huaxingao (via GitHub)" <gi...@apache.org> on 2023/12/02 04:05:55 UTC

[PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

huaxingao opened a new pull request, #44119:
URL: https://github.com/apache/spark/pull/44119

   
   
   ### What changes were proposed in this pull request?
   Add `MergeInto` support in `DataFrameWriterV2`
   
   
   ### Why are the changes needed?
   Spark currently supports merge into sql statement. We want DataFrame to have the same support.
   
   ### Does this PR introduce _any_ user-facing change?
   Yes. This PR introduces new API like the following:
   
   ```
         spark.table("source")
           .mergeInto("target")
           .on($"source.id" === $"target.id")
           .whenNotMatched()
           .insertAll()
           .merge()
   ```
   
   
   ### How was this patch tested?
   new tests
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   No
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431044881


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {

Review Comment:
   ```suggestion
   case class WhenMatched[T] private(mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431049042


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {

Review Comment:
   why do we have this base suite if it only has one sub-class?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432297831


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.

Review Comment:
   Delete the matching target table 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer closed pull request #44119: [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2
URL: https://github.com/apache/spark/pull/44119


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432307906


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.

Review Comment:
   This is user facing API doc. Not sure if it is proper to put `MergeIntoWriter` there. For example, we don't put `DataFrameWriter` in `write` API 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424923942


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)

Review Comment:
   Is `whenNotMatchedBySource` simply `whenNotMatched` plus a source condition? If so, why it isn't `whenNotMatched($"salary" === 100)` like `whenMatched` with condition?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on PR #44119:
URL: https://github.com/apache/spark/pull/44119#issuecomment-1852435665

   cc @aokolnychyi @cloud-fan @dongjoon-hyun @viirya 
   This PR is ready for review. Could you please take a look when you have a moment? Thanks a lot!
   There is still a mima check failure. It passed on my local, though. I will figure out what is wrong.
   


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431047334


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {

Review Comment:
   ditto, should use private constructor



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on PR #44119:
URL: https://github.com/apache/spark/pull/44119#issuecomment-1865382871

   Merged to master.
   Thank you @huaxingao @cloud-fan @viirya 


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429448755


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")

Review Comment:
   I think compile-time guarantee is better than runtime check, but I'm open to other ideas.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430965633


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.

Review Comment:
   do you mean MERGE 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "HyukjinKwon (via GitHub)" <gi...@apache.org>.
HyukjinKwon commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1413295966


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,63 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  def on(condition: String): DataFrameWriterV2[T] = {

Review Comment:
   Would need the Scaladoc.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432308805


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.

Review Comment:
   Maybe just describing what the function is used for. E.g., "Merges a set of updates, insertions, and deletions based on a source table into a target table"
   
   https://docs.databricks.com/en/sql/language-manual/delta-merge-into.html



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431313594


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {
+
+  import testImplicits._
+
+  test("merge into schema pruning with WHEN MATCHED clause (update)") {

Review Comment:
   Can we check the commit history? Maybe this was a mistake that we should not follow.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431953737


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](

Review Comment:
   I want to keep `mergeIntoWriter` simple. I prefer to keep the current implementation if you don't mind.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430964633


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {

Review Comment:
   what APIs do we inherit from `DataFrameWriterV2`?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431082677


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()

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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431047091


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)

Review Comment:
   BTW, I think we can make the code simpler by adding a new method in `MergeIntoWriter`:
   ```
   def withNewMatchedAction(action): MergeIntoWriter = {
     this.matchedActions = this.matchedActions :+ ...
     this
   }
   ```
   
   and here we just need to do
   ```
   mergeIntoWriter.withNewMatchedAction(...)
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424908372


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    // TODO: streaming could be adapted to use this interface

Review Comment:
   Hm, does MergeInto support structured streaming?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1433027116


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2726,6 +2726,12 @@
     ],
     "sqlState" : "42000"
   },
+  "NO_MERGE_ACTION_SPECIFIED" : {
+    "message" : [
+      "df.mergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource."
+    ],
+    "sqlState" : "23K02"

Review Comment:
   Fixed. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+
+  def withNewMatchedUpdateAction(condition: Option[Expression]): MergeIntoWriter[T] = {

Review Comment:
   Fixed. Thanks



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1427185535


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)

Review Comment:
   Checks are added



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.

Review Comment:
   Fixed. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {

Review Comment:
   Yes, I fixed the doc to make it more clear



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    val merge = MergeIntoTable(

Review Comment:
   Check is added



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428618073


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.

Review Comment:
   Ideally we don't need this overload. People can just cal `expr(string)` to do the same thing, and we keep the API clean.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431038746


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {

Review Comment:
   Actually doesn't need the inheritance



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.

Review Comment:
   Fixed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431042435


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)

Review Comment:
   Each invocation of `mergeInto` would create a new instance of `MergeIntoWriter`.  I think we don't need to consider thread safe 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430581901


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -36,7 +37,7 @@ import org.apache.spark.sql.types.IntegerType
  */
 @Experimental
 final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
-    extends CreateTableWriter[T] {
+    extends CreateTableWriter[T] with MergeIntoWriter[T] {

Review Comment:
   Changed `MergeIntoWriter` to a concrete class.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430966684


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.

Review Comment:
   ditto



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430967389


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])

Review Comment:
   can we put the merge condition in the constructor? then we can remove the `def on`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432297510


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.

Review Comment:
   Delete the target table 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432675303


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.

Review Comment:
   The key is the `MergeIntoWriter` is public API or developer API.
   cc @cloud-fan 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424909535


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.

Review Comment:
   ```suggestion
      * Create a [[DataFrameWriterV2]] for MergeInto action.
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428621468


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    if (isStreaming) {
+      logicalPlan.failAnalysis(
+        errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED",
+        messageParameters = Map("methodName" -> toSQLId("mergeInto")))
+    }
+    new DataFrameWriterV2[T](table, this)

Review Comment:
   I think it's a good place here to enter the merge API layer, e.g. `MergeIntoBuilder[T]`, similar to `CreateTableWriter[T]`. It's still the `DataFrameWriterV2` to extend `MergeIntoBuilder[T]`, but the exposed APIs become a subset.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1433027955


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.

Review Comment:
   Fixed. Thanks



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432295630


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.

Review Comment:
   Insert the specified 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on PR #44119:
URL: https://github.com/apache/spark/pull/44119#issuecomment-1865358667

   The test failure doesn't seem to be related to my changes.


-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429683995


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -36,7 +37,7 @@ import org.apache.spark.sql.types.IntegerType
  */
 @Experimental
 final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
-    extends CreateTableWriter[T] {
+    extends CreateTableWriter[T] with MergeIntoWriter[T] {

Review Comment:
   is there anything shared with existing `DataFrameWriterV2` functions? If not maybe we can just add a new concrete class `MergeIntoWriter`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431049589


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {
+
+  import testImplicits._
+
+  test("merge into schema pruning with WHEN MATCHED clause (update)") {

Review Comment:
   what's special about the tests here? why do they only work in delta based v2 sources?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429453968


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,59 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  def whenMatched(): WhenMatched[T] = {

Review Comment:
   in SQL, users can't put `WHEN MATCHED` tiwce. Can we enforce it in the scala API as well? e.g.
   ```
   df.mergeInto...
   .whenMatched...
   .whenMatched... // runtime error
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430967797


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "MERGE_INTO_API_ERROR" : {
+    "message" : [
+      "MergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource."

Review Comment:
   ```suggestion
         "df.mergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource."
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424912156


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.

Review Comment:
   ```suggestion
      * Initialize a `WhenNotMatchedBySource` action with a condition.
   ```
   



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428619217


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")

Review Comment:
   if this is required, it should be provided at the entry point, e.g. `df.mergeInto(target, onCondition)`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429394708


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    if (isStreaming) {

Review Comment:
   After introducing a new `MergeIntoWriter`, it became `def mergeInto(table: String): MergeIntoWriter[T]`



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {

Review Comment:
   Added `MergeIntoWriter[T]`



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.

Review Comment:
   Removed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431039100


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.

Review Comment:
   Fixed



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])

Review Comment:
   Fixed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431042933


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]

Review Comment:
   these should be private fields? or at lease private[sql]



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431301010


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the insert all action configured.
+   */
+  def insertAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions =
+      mergeIntoWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The MergeIntoWriter instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): MergeIntoWriter[T] = {

Review Comment:
   `set` -> `map`?



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a

Review Comment:
   `DMergeIntoWriter` ?



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a

Review Comment:
   `DMergeIntoWriter` ?



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](

Review Comment:
   How about defined as inner class, so we can remove `mergeIntoWriter` here?
   Of course, the current implementation looks good too.



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the insert all action configured.
+   */
+  def insertAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions =
+      mergeIntoWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The MergeIntoWriter instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions = mergeIntoWriter.notMatchedActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be performed when there is no match by source
+ * during a merge operation in a MergeIntoWriter.
+ *
+ * @param MergeIntoWriter the MergeIntoWriter instance to which the merge actions will be applied.
+ * @param condition       an optional condition to be used with the merge actions.
+ * @tparam T              the type parameter for the MergeIntoWriter.
+ */
+case class WhenNotMatchedBySource[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to update all non-matched rows in the target DataFrame when
+   * not matched by the source.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedBySourceActions =
+      mergeIntoWriter.notMatchedBySourceActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {

Review Comment:
   ditto.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432291759


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)

Review Comment:
   Oh, I see. `whenNotMatched` is actually "WHEN NOT MATCHED [BY TARGET]".



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432275112


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2726,6 +2726,12 @@
     ],
     "sqlState" : "42000"
   },
+  "NO_MERGE_ACTION_SPECIFIED" : {
+    "message" : [
+      "df.mergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource."
+    ],
+    "sqlState" : "23K02"

Review Comment:
   ```suggestion
       "sqlState" : "42K0E"
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on PR #44119:
URL: https://github.com/apache/spark/pull/44119#issuecomment-1865384190

   Thank you all very much for reviewing the 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432296781


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only

Review Comment:
   Update the specified columns of the target 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424921268


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    val merge = MergeIntoTable(

Review Comment:
   Do we need to check if `on` is specified (i.e., `Some`)?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432277061


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala:
##########
@@ -0,0 +1,946 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.functions._
+
+class MergeIntoDataFrameSuite extends RowLevelOperationSuiteBase {

Review Comment:
   +1. We only need to make sure the new scala API works. We don't need to test the underlying v2 sources extensively.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424909883


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +572,84 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {

Review Comment:
   Some class docs for these classes should be good to add.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428621468


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    if (isStreaming) {
+      logicalPlan.failAnalysis(
+        errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED",
+        messageParameters = Map("methodName" -> toSQLId("mergeInto")))
+    }
+    new DataFrameWriterV2[T](table, this)

Review Comment:
   I think it's a good place here to enter the merge API layer, e.g. `MergeIntoWriter[T]`, similar to `CreateTableWriter[T]`. It's still the `DataFrameWriterV2` to extend `MergeIntoWriter[T]`, but the exposed APIs become a subset.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431040535


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "NO_MERGE_ACTION_ERROR" : {

Review Comment:
   `NO_MERGE_ACTION_SPECIFIED`, I updated my review comment and you probably missed the update...



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430966112


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)

Review Comment:
   do we need to consider thread safety 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429453968


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,59 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  def whenMatched(): WhenMatched[T] = {

Review Comment:
   in SQL, users can't put `WHEN MATCHED` tiwce. Can we enforce it in the scala API as well? e.g.
   ```
   df.mergeInto...
   .whenMatched...
   .whenMatched... // runtime error
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429395189


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")

Review Comment:
   I still think 
   
   ```
         spark.table("source")
           .mergeInto("target")
           .on($"source.id" === $"target.id")
           .whenNotMatched()
           .insertAll()
           .merge()
   ```
   might be a bit better than
   ```
         spark.table("source")
           .mergeInto("target", $"source.id" === $"target.id")
           .whenNotMatched()
           .insertAll()
           .merge()
   ```
   
   But I will change if you prefer the second one.



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    if (isStreaming) {
+      logicalPlan.failAnalysis(
+        errorClass = "CALL_ON_STREAMING_DATASET_UNSUPPORTED",
+        messageParameters = Map("methodName" -> toSQLId("mergeInto")))
+    }
+    new DataFrameWriterV2[T](table, this)

Review Comment:
   Added a new trait `MergeIntoWriter`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429453554


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +402,312 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+trait MergeIntoWriter[T] {
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T]
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   should be applied.
+ *                   If the condition is None, the actions will be applied to all matched rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   defined in this configuration should be applied.
+ *                   If the condition is None, the actions will be applied when there
+ *                   are no matching rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenNotMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all non-matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the insert all action configured.
+   */
+  def insertAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The DataFrameWriterV2 instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete non-matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be performed when there is no match by source
+ * during a merge operation in a DataFrameWriterV2.
+ *
+ * @param dfWriter the DataFrameWriterV2 instance to which the merge actions will be applied.
+ * @param condition an optional condition to be used with the merge actions.
+ * @tparam T the type parameter for the DataFrameWriterV2.
+ */
+case class WhenNotMatchedBySource[T] (
+    dfWriter: DataFrameWriterV2[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to update all non-matched rows in the target DataFrame when
+   * not matched by the source.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions =
+      dfWriter.notMatchedBySourceActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions = dfWriter.notMatchedBySourceActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert all non-matched rows into the target DataFrame when not
+   * matched by the source.
+   *
+   * @return The DataFrameWriterV2 instance with the insert all action configured.
+   */
+  def insertAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions =
+      dfWriter.notMatchedBySourceActions :+ InsertStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The DataFrameWriterV2 instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions = dfWriter.notMatchedBySourceActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete non-matched rows from the target DataFrame when not matched by
+   * the source.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {

Review Comment:
   ditto, the SQL grammar does not have DELETE clause in `WHEN NOT MATCHED BY SOURCE`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424911090


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` object with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` object with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)

Review Comment:
   Do we need some sanity checks for these parameters to `MergeIntoTable`? I.e., `matchedActions`, `notMatchedActions`, `notMatchedBySourceActions`? For example, can they all empty etc.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1427185349


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    // TODO: streaming could be adapted to use this interface

Review Comment:
   Removed. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,37 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for mergeInto action.

Review Comment:
   Fixed. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +572,84 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {

Review Comment:
   Class docs are added. Thanks



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432301555


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+
+  def withNewMatchedUpdateAction(condition: Option[Expression]): MergeIntoWriter[T] = {
+    this.matchedActions = this.matchedActions :+ UpdateStarAction(condition)
+    this
+  }
+
+  def withNewMatchedUpdateAction(
+      condition: Option[Expression],
+      map: Map[String, Column]): MergeIntoWriter[T] = {
+    this.matchedActions = this.matchedActions :+
+      UpdateAction(condition, map.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this
+  }

Review Comment:
   Hmm, these withXXX methods will be exposed too? Seems they can be private to spark at least (not exposed)?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430968156


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "MERGE_INTO_API_ERROR" : {

Review Comment:
   ```suggestion
     "NO_MERGE_ACTION_SPECIFIED" : {
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431953526


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a

Review Comment:
   Fixed. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the insert all action configured.
+   */
+  def insertAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions =
+      mergeIntoWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The MergeIntoWriter instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): MergeIntoWriter[T] = {

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,315 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the insert all action configured.
+   */
+  def insertAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions =
+      mergeIntoWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The MergeIntoWriter instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedActions = mergeIntoWriter.notMatchedActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be performed when there is no match by source
+ * during a merge operation in a MergeIntoWriter.
+ *
+ * @param MergeIntoWriter the MergeIntoWriter instance to which the merge actions will be applied.
+ * @param condition       an optional condition to be used with the merge actions.
+ * @tparam T              the type parameter for the MergeIntoWriter.
+ */
+case class WhenNotMatchedBySource[T] private[sql](
+    mergeIntoWriter: MergeIntoWriter[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to update all non-matched rows in the target DataFrame when
+   * not matched by the source.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.notMatchedBySourceActions =
+      mergeIntoWriter.notMatchedBySourceActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {

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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432306542


##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -431,7 +431,17 @@ object CheckConnectJvmClientCompatibility {
 
       // Encoders are in the wrong JAR
       ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders"),
-      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$"))
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.SQLImplicits._sqlContext"),

Review Comment:
   Hm, do you change `SQLImplicits._sqlContext`?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432273906


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {

Review Comment:
   SGTM!



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432284747


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target", $"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0

Review Comment:
   Similar to `def write`, `@group basic`? 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431081337


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "NO_MERGE_ACTION_ERROR" : {

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.

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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431084724


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {

Review Comment:
   Trying to follow the structure of the original test suites



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428617043


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {

Review Comment:
   Shall we add a new layer for merge APIs? e.g. `partitionedBy` returns `CreateTableWriter`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1424920459


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,229 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` object without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` object with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {

Review Comment:
   The case that condition is a string means it is a column 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432276335


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+
+  def withNewMatchedUpdateAction(condition: Option[Expression]): MergeIntoWriter[T] = {

Review Comment:
   this seems too many helper functions...
   ```
   def withNewMatchedAction(action: MergeAction): MergeIntoWriter[T] = {
     this.matchedActions = this.matchedActions :+ action
     this
   }
   ```
   I think 3 helper functions should be good enough for 3 different action types.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1433027599


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new SparkRuntimeException(
+        errorClass = "NO_MERGE_ACTION_SPECIFIED",
+        messageParameters = Map.empty)
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+
+  def withNewMatchedUpdateAction(condition: Option[Expression]): MergeIntoWriter[T] = {
+    this.matchedActions = this.matchedActions :+ UpdateStarAction(condition)
+    this
+  }
+
+  def withNewMatchedUpdateAction(
+      condition: Option[Expression],
+      map: Map[String, Column]): MergeIntoWriter[T] = {
+    this.matchedActions = this.matchedActions :+
+      UpdateAction(condition, map.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this
+  }

Review Comment:
   Fixed



##########
connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/CheckConnectJvmClientCompatibility.scala:
##########
@@ -431,7 +431,17 @@ object CheckConnectJvmClientCompatibility {
 
       // Encoders are in the wrong JAR
       ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders"),
-      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$"))
+      ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.Encoders$"),
+      ProblemFilters.exclude[Problem]("org.apache.spark.sql.SQLImplicits._sqlContext"),

Review Comment:
   removed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1433027290


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[MergeIntoWriter]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target", $"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0

Review Comment:
   Added. Thanks



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.

Review Comment:
   I have fixed this and a few other places



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429645287


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,241 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition, provided as a `String`, to be used for merging data. This condition
+   * is converted internally to a `Column` and used to determine how rows from the source
+   * DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `String` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: String): DataFrameWriterV2[T] = {
+    on(Column(condition))
+  }
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): DataFrameWriterV2[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a specified condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the given `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * The `WhenMatched` instance can perform one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: String): WhenMatched[T] = {
+    whenMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: String): WhenNotMatched[T] = {
+    whenNotMatched(Column(condition))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied. The condition is represented as a `String` and internally converted
+   * to a `Column`.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `String` representing a column name which specifies the condition
+   *                  to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: String): WhenNotMatchedBySource[T] = {
+    whenNotMatchedBySource(Column(condition))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")

Review Comment:
   Changed to `mergeInto(table: String, condition: Column)`



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,59 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")
+    }
+
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new IllegalStateException("At least one of matchedActions, notMatchedActions," +

Review Comment:
   new error was added



##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +402,312 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+trait MergeIntoWriter[T] {
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T]
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   should be applied.
+ *                   If the condition is None, the actions will be applied to all matched rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   defined in this configuration should be applied.
+ *                   If the condition is None, the actions will be applied when there
+ *                   are no matching rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenNotMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {

Review Comment:
   Update and Delete were removed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429645585


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +402,312 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+trait MergeIntoWriter[T] {
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T]
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   should be applied.
+ *                   If the condition is None, the actions will be applied to all matched rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   defined in this configuration should be applied.
+ *                   If the condition is None, the actions will be applied when there
+ *                   are no matching rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenNotMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all non-matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert all non-matched rows into the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the insert all action configured.
+   */
+  def insertAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ InsertStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the DataFrame with the provided
+   * column assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The DataFrameWriterV2 instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete non-matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedActions = dfWriter.notMatchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be performed when there is no match by source
+ * during a merge operation in a DataFrameWriterV2.
+ *
+ * @param dfWriter the DataFrameWriterV2 instance to which the merge actions will be applied.
+ * @param condition an optional condition to be used with the merge actions.
+ * @tparam T the type parameter for the DataFrameWriterV2.
+ */
+case class WhenNotMatchedBySource[T] (
+    dfWriter: DataFrameWriterV2[T],
+    condition: Option[Expression]) {
+
+  /**
+   * Specifies an action to update all non-matched rows in the target DataFrame when
+   * not matched by the source.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions =
+      dfWriter.notMatchedBySourceActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update non-matched rows in the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions = dfWriter.notMatchedBySourceActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert all non-matched rows into the target DataFrame when not
+   * matched by the source.
+   *
+   * @return The DataFrameWriterV2 instance with the insert all action configured.
+   */
+  def insertAll(): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions =
+      dfWriter.notMatchedBySourceActions :+ InsertStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to insert non-matched rows into the target DataFrame with the provided
+   * column assignments when not matched by the source.
+   *
+   * @param set A Map of column names to Column expressions representing the values to be inserted.
+   * @return The DataFrameWriterV2 instance with the insert action configured.
+   */
+  def insert(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.notMatchedBySourceActions = dfWriter.notMatchedBySourceActions :+
+      InsertAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete non-matched rows from the target DataFrame when not matched by
+   * the source.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {

Review Comment:
   Insert was removed



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429449276


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -167,6 +173,59 @@ final class DataFrameWriterV2[T] private[sql](table: String, ds: Dataset[T])
     runCommand(overwrite)
   }
 
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (on.isEmpty) {
+      throw new IllegalStateException("The 'on' condition cannot be None")
+    }
+
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw new IllegalStateException("At least one of matchedActions, notMatchedActions," +

Review Comment:
   nit: this is a user error, let's add an error class for 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431768750


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {

Review Comment:
   The original test suites have:
   
   ```
   /* This suite only adds tests specific to group-based MERGE operations, 
    * https://github.com/apache/spark/pull/41577
    * most of the tests are in the parent class.
    * /
   class GroupBasedMergeIntoTableSuite extends MergeIntoTableSuiteBase
   
   /* This suite only adds tests specific to delta-based MERGE operations
    * https://github.com/apache/spark/pull/41930. 
    * For the MERGE operation in this test suite, we only need to read the required columns from 
    * the target table, but for group-based MERGE operations, we need to read all the columns
    * from the target table. 
    * /
   abstract class DeltaBasedMergeIntoTableSuiteBase extends MergeIntoTableSuiteBase
   ```
   
   ```
   
   class DeltaBasedMergeIntoTableSuite extends DeltaBasedMergeIntoTableSuiteBase
   //    props.put("supports-deltas", "true")
   
   class DeltaBasedMergeIntoTableUpdateAsDeleteAndInsertSuite extends DeltaBasedMergeIntoTableSuiteBase
   //    props.put("supports-deltas", "true")
   //    props.put("split-updates", "true")
   ```
   
   I feel for DataFrame Merge API support, I don't need all these test suites, but I can follow the original tests to add all the corresponding test suites.
   
   



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431953328


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,316 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T])
+  extends DataFrameWriterV2[T](table, ds) {
+
+  var on: Option[Column] = None
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `MergeIntoWriter` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T] = {
+    this.on = Some(condition)
+    this
+  }
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.get.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)

Review Comment:
   I didn't see this last night. Fixed just now.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431044053


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()

Review Comment:
   And this should be in `QeuryCompilationErrors`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1429453151


##########
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriterV2.scala:
##########
@@ -343,3 +402,312 @@ trait CreateTableWriter[T] extends WriteConfigMethods[CreateTableWriter[T]] {
    */
   def tableProperty(property: String, value: String): CreateTableWriter[T]
 }
+
+trait MergeIntoWriter[T] {
+
+  /**
+   * Specifies the merge condition.
+   *
+   * Sets the condition to be used for merging data. This condition is used to determine
+   * how rows from the source DataFrame are matched with rows in the target table.
+   *
+   * @param condition a `Column` representing the merge condition.
+   * @return the current `DataFrameWriterV2` instance with the specified merge condition set.
+   */
+  def on(condition: Column): MergeIntoWriter[T]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T]
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T]
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   should be applied.
+ *                   If the condition is None, the actions will be applied to all matched rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the update all action configured.
+   */
+  def updateAll(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ UpdateStarAction(condition)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The DataFrameWriterV2 instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.dfWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The DataFrameWriterV2 instance with the delete action configured.
+   */
+  def delete(): DataFrameWriterV2[T] = {
+    dfWriter.matchedActions = dfWriter.matchedActions :+ DeleteAction(condition)
+    this.dfWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during an update operation.
+ *
+ * @param dfWriter   The DataFrameWriterV2 instance responsible for writing data to a
+ *                   target DataFrame.
+ * @param condition  An optional condition Expression that specifies when the actions
+ *                   defined in this configuration should be applied.
+ *                   If the condition is None, the actions will be applied when there
+ *                   are no matching rows.
+ *
+ * @tparam T         The type of data in the DataFrame.
+ */
+case class WhenNotMatched[T] (dfWriter: DataFrameWriterV2[T], condition: Option[Expression]) {

Review Comment:
   I checked the SQL grammar, WHEN NOT MATCHED only supports INSERT clause, which makes sense as there is no target row in the target table to update/delete.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432296302


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.

Review Comment:
   `Update all the columns of the target 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432294819


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.

Review Comment:
   `Insert all the columns of the target table with ....`?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "viirya (via GitHub)" <gi...@apache.org>.
viirya commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1433054099


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.

Review Comment:
   Yea, I was referring the doc too. I've tried to combine @huaxingao original sentence and the doc. If @huaxingao wants to use these description from the doc, it is good too.



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.

Review Comment:
   Thanks.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431039370


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "MERGE_INTO_API_ERROR" : {
+    "message" : [
+      "MergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource."

Review Comment:
   Done



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "MERGE_INTO_API_ERROR" : {

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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431083576


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {

Review Comment:
   Done



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()
+    }
+
+    val merge = MergeIntoTable(
+      UnresolvedRelation(tableName),
+      logicalPlan,
+      on.expr,
+      matchedActions,
+      notMatchedActions,
+      notMatchedBySourceActions)
+    val qe = sparkSession.sessionState.executePlan(merge)
+    qe.assertCommandExecuted()
+  }
+}
+
+/**
+ * A class for defining actions to be taken when matching rows in a DataFrame during
+ * a merge operation.
+ *
+ * @param mergeIntoWriter   The MergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          should be applied.
+ *                          If the condition is None, the actions will be applied to all matched
+ *                          rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {
+  /**
+   * Specifies an action to update all matched rows in the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the update all action configured.
+   */
+  def updateAll(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ UpdateStarAction(condition)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to update matched rows in the DataFrame with the provided column
+   * assignments.
+   *
+   * @param set A Map of column names to Column expressions representing the updates to be applied.
+   * @return The MergeIntoWriter instance with the update action configured.
+   */
+  def update(set: Map[String, Column]): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+
+      UpdateAction(condition, set.map(x => Assignment(expr(x._1).expr, x._2.expr)).toSeq)
+    this.mergeIntoWriter
+  }
+
+  /**
+   * Specifies an action to delete matched rows from the DataFrame.
+   *
+   * @return The MergeIntoWriter instance with the delete action configured.
+   */
+  def delete(): MergeIntoWriter[T] = {
+    mergeIntoWriter.matchedActions = mergeIntoWriter.matchedActions :+ DeleteAction(condition)
+    this.mergeIntoWriter
+  }
+}
+
+/**
+ * A class for defining actions to be taken when no matching rows are found in a DataFrame
+ * during a merge operation.
+ *
+ * @param MergeIntoWriter   The DMergeIntoWriter instance responsible for writing data to a
+ *                          target DataFrame.
+ * @param condition         An optional condition Expression that specifies when the actions
+ *                          defined in this configuration should be applied.
+ *                          If the condition is None, the actions will be applied when there
+ *                          are no matching rows.
+ *
+ * @tparam T                The type of data in the MergeIntoWriter.
+ */
+case class WhenNotMatched[T] (mergeIntoWriter: MergeIntoWriter[T], condition: Option[Expression]) {

Review Comment:
   Done



##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {
+
+  import testImplicits._
+
+  test("merge into schema pruning with WHEN MATCHED clause (update)") {

Review Comment:
   Actually i am not sure, just following the original test suite `DeltaBasedMergeIntoTableSuiteBase`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432277061


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/MergeIntoDataFrameSuite.scala:
##########
@@ -0,0 +1,946 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.functions._
+
+class MergeIntoDataFrameSuite extends RowLevelOperationSuiteBase {

Review Comment:
   +1. We only need to make sure the new scala API works. We don't need to test the underlying v2 sources extensively, which should have been covered already by other tests



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1430968156


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2495,6 +2495,11 @@
     ],
     "sqlState" : "23K01"
   },
+  "MERGE_INTO_API_ERROR" : {

Review Comment:
   ```suggestion
     "NO_MERGE_ACTION_ERROR" : {
   ```



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431043833


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()

Review Comment:
   can we rename this method to match the error class 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431042358


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.

Review Comment:
   let's doc the merge condition 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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "beliefer (via GitHub)" <gi...@apache.org>.
beliefer commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1432680302


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.

Review Comment:
   How about `Delete all target rows that have a match in the source table.`?
   Please refer https://docs.databricks.com/en/sql/language-manual/delta-merge-into.html#when-matched



##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,350 @@
+/*
+ * 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
+
+import org.apache.spark.SparkRuntimeException
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ * @param on the merge condition.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  private[sql] var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  private[sql] var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.

Review Comment:
   `Insert all rows from the source that are not already in the target table.`
   Please refer https://docs.databricks.com/en/sql/language-manual/delta-merge-into.html#when-not-matched-[by-target]



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431044641


##########
sql/core/src/main/scala/org/apache/spark/sql/MergeIntoWriter.scala:
##########
@@ -0,0 +1,308 @@
+/*
+ * 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
+
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction}
+import org.apache.spark.sql.errors.QueryExecutionErrors
+import org.apache.spark.sql.functions.expr
+
+/**
+ * `MergeIntoWriter` provides methods to define and execute merge actions based
+ * on specified conditions.
+ *
+ * @tparam T the type of data in the Dataset.
+ * @param table the name of the target table for the merge operation.
+ * @param ds the source Dataset to merge into the target table.
+ *
+ * @since 4.0.0
+ */
+@Experimental
+class MergeIntoWriter[T] private[sql] (table: String, ds: Dataset[T], on: Column) {
+
+  private val df: DataFrame = ds.toDF()
+
+  private val sparkSession = ds.sparkSession
+
+  private val tableName = sparkSession.sessionState.sqlParser.parseMultipartIdentifier(table)
+
+  private val logicalPlan = df.queryExecution.logical
+
+  var matchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedActions: Seq[MergeAction] = Seq.empty[MergeAction]
+  var notMatchedBySourceActions: Seq[MergeAction] = Seq.empty[MergeAction]
+
+  /**
+   * Initialize a `WhenMatched` action without any condition.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenMatched` object.
+   */
+  def whenMatched(): WhenMatched[T] = {
+    new WhenMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenMatched` action with a condition.
+   *
+   * This `WhenMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenMatched` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenMatched` object configured with the specified condition.
+   */
+  def whenMatched(condition: Column): WhenMatched[T] = {
+    new WhenMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action without any condition.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @return a new `WhenNotMatched` object.
+   */
+  def whenNotMatched(): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatched` action with a condition.
+   *
+   * This `WhenNotMatched` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatched` can be followed by one of the following merge actions:
+   *   - `insertAll`: Insert all the target table with source dataset records.
+   *   - `insert(Map)`: Insert all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatched` object configured with the specified condition.
+   */
+  def whenNotMatched(condition: Column): WhenNotMatched[T] = {
+    new WhenNotMatched[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action without any condition.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @return a new `WhenNotMatchedBySource` object.
+   */
+  def whenNotMatchedBySource(): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, None)
+  }
+
+  /**
+   * Initialize a `WhenNotMatchedBySource` action with a condition.
+   *
+   * This `WhenNotMatchedBySource` action will be executed if and only if the specified `condition`
+   * is satisfied.
+   *
+   * This `WhenNotMatchedBySource` can be followed by one of the following merge actions:
+   *   - `updateAll`: Update all the target table fields with source dataset fields.
+   *   - `update(Map)`: Update all the target table records while changing only
+   *     a subset of fields based on the provided assignment.
+   *   - `delete`: Delete all the target table records.
+   *
+   * @param condition a `Column` representing the condition to be evaluated for the action.
+   * @return a new `WhenNotMatchedBySource` object configured with the specified condition.
+   */
+  def whenNotMatchedBySource(condition: Column): WhenNotMatchedBySource[T] = {
+    new WhenNotMatchedBySource[T](this, Some(condition.expr))
+  }
+
+  /**
+   * Executes the merge operation.
+   */
+  def merge(): Unit = {
+    if (matchedActions.isEmpty && notMatchedActions.isEmpty && notMatchedBySourceActions.isEmpty) {
+      throw QueryExecutionErrors.mergeIntoAPIError()

Review Comment:
   Or just inline it: `throw new SparkException(...)`



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "huaxingao (via GitHub)" <gi...@apache.org>.
huaxingao commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1431780938


##########
sql/core/src/test/scala/org/apache/spark/sql/connector/DeltaBasedMergeIntoDFWriterV2SuiteBase.scala:
##########
@@ -0,0 +1,224 @@
+/*
+ * 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.connector
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.types.DataTypeUtils
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.execution.datasources.v2.BatchScanExec
+import org.apache.spark.sql.functions.{col, lit}
+import org.apache.spark.sql.types.StructType
+
+abstract class DeltaBasedMergeIntoDFWriterV2SuiteBase extends MergeIntoDFWriterV2SuiteBase {

Review Comment:
   @cloud-fan I think over, since i only need to test the newly added DataFrame Merge API support, I will just keep `MergeIntoDFWriterV2SuiteBase` and rename it to `MergeIntoDataFrameSuite`. This should be sufficient. I will delete `DeltaBasedMergeIntoDFWriterV2Suite` and `DeltaBasedMergeIntoDFWriterV2SuiteBase`.



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


Re: [PR] [SPARK-46207][SQL] Support MergeInto in DataFrameWriterV2 [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #44119:
URL: https://github.com/apache/spark/pull/44119#discussion_r1428615608


##########
sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala:
##########
@@ -4129,6 +4129,36 @@ class Dataset[T] private[sql](
     new DataFrameWriterV2[T](table, this)
   }
 
+  /**
+   * Create a [[DataFrameWriterV2]] for MergeInto action.
+   *
+   * Scala Examples:
+   * {{{
+   *   spark.table("source")
+   *     .mergeInto("target")
+   *     .on($"source.id" === $"target.id")
+   *     .whenMatched($"salary" === 100)
+   *     .delete()
+   *     .whenNotMatched()
+   *     .insertAll()
+   *     .whenNotMatchedBySource($"salary" === 100)
+   *     .update(Map(
+   *       "salary" -> lit(200)
+   *     ))
+   *     .merge()
+   * }}}
+   *
+   * @since 4.0.0
+   */
+  def mergeInto(table: String): DataFrameWriterV2[T] = {
+    if (isStreaming) {

Review Comment:
   the implementation looks identical to `writeTo`?



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org