Closed sh0ck-wave closed 3 months ago
The issue is we don't call Dataset.ofRows for Merge: https://github.com/delta-io/delta/blob/c2b0ce6d4f35df2fe6c2281299ae38513f50305d/core/src/main/scala/io/delta/tables/DeltaMergeBuilder.scala#L264
There were some spark issues preventing us from doing this. Would you be willing to help us to try out and see if these spark issues have got fixed?
There were some spark issues preventing us from doing this. Would you be willing to help us to try out and see if these spark issues have got fixed?
Yes, Can you provide some guidance on how to repro these issues
Yes, Can you provide some guidance on how to repro these issues
You can call toDataset on the merge command like https://github.com/delta-io/delta/blob/edaeb86304211513c8028d056a7d90e98ec2839c/core/src/main/scala/io/delta/tables/execution/DeltaTableOperations.scala#L80 and see if there is any test failed. If Spark hasn't fixed the issue, there will be tests failing.
@zsxwing spark have fixed the issue . spark issue Previously, I made these changes in a pull request and the tests passed. should I create a pull request for this change
should I create a pull request for this change
Yep. Feel free to open a pull request.
@sherlockbeard thanks for working on this item @zsxwing I can confirm that the PR solves this Bug. Testing the PR, I was able to get the following output with it from the same sample app as above:
Captured plans for SQL MERGE:
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Aggregate
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
Received Event from class org.apache.spark.sql.catalyst.plans.logical.GlobalLimit
Received Event from class org.apache.spark.sql.delta.commands.MergeIntoCommand
Captured plans for Delta API:
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Aggregate
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Project
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
Received Event from class org.apache.spark.sql.catalyst.plans.logical.Filter
Received Event from class org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject
Received Event from class org.apache.spark.sql.catalyst.plans.logical.GlobalLimit
Received Event from class org.apache.spark.sql.delta.commands.MergeIntoCommand
@sh0ck-wave @sherlockbeard Hi, appreciate the effort you put into resolving and researching this. Is there a timeline for when this fix can be merged? Or is there something check that's blocking it's release/approval?
I picked up the change from @sherlockbeard, ran some more tests and merged it: https://github.com/delta-io/delta/pull/3456 cc @fluxquantum
Hi johanl, really appreciate the update. Awesome on the quick turnaround. I know I posed this question in an earlier thread. Is there a version of the library I can pull as a patch for now or will I need to wait for a major release?
Bug
MergeIntoCommand not visible in QueryExecutionListener when using Python/Scala API to execute a merge operation
Describe the problem
When using sql MERGE statement via
spark.sql
a LogicalPlan of typeorg.apache.spark.sql.delta.commands.MergeIntoCommand
is visible to any registered spark QueryExecutionListener, this is useful for capturing statistics and data lineage. When using the python API to execute the merge operation, no such LogicalPlan is visible to registered spark QueryExecutionListeners making it difficult to track merge related statistics and data lineageSteps to reproduce
Execute the following scala spark application:
Observed results
As can be seen in the case of Delta API there is no
org.apache.spark.sql.delta.commands.MergeIntoCommand
captured by the QueryExecutionListenerExpected results
org.apache.spark.sql.delta.commands.MergeIntoCommand
should be captured by QueryExecutionListener for Delta API similar to SQL MERGE commandEnvironment information