-
Notifications
You must be signed in to change notification settings - Fork 3k
Spark 3.2: Implement merge-on-read MERGE #4047
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Conversation
|
|
||
| sql("ALTER TABLE %s ADD PARTITION FIELD dep", tableName); | ||
|
|
||
| append(tableName, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I had to adapt this test and use a similar trick like in a few other places where we delete files we don't expect to query. Otherwise, I'd need to have different checks for copy-on-write and merge-on-read.
da69be2 to
7b88c94
Compare
7b88c94 to
fea1dd9
Compare
| // merge mode is NOT SET -> rely on write distribution and ordering as a basis | ||
| // merge mode is NONE -> unspecified distribution + | ||
| // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, days(ts) | ||
| // merge mode is HASH -> CLUSTER BY _spec_id, _partition, date, days(ts) + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
This means we may split records for a partition into a number of tasks if that partition is in the old spec and is not aligned with the current partitioning (i.e. date, days(ts) in this case). Not sure how to avoid this, though.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I just thought about this for the case above. I think it's okay.
| // LOCALLY ORDERED BY _spec_id, _partition, _file, _pos, date, id | ||
| // merge mode is HASH -> CLUSTER BY _spec_id, _partition, date + | ||
| // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, date, id | ||
| // merge mode is RANGE -> RANGE DISTRIBUTE BY _spec_id, _partition, _file, date, id |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note that we don't include _pos in the distribution. Just in the local ordering.
...extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
Outdated
Show resolved
Hide resolved
...extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
Outdated
Show resolved
Hide resolved
...extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
Show resolved
Hide resolved
spark/v3.2/spark-extensions/src/test/java/org/apache/iceberg/spark/extensions/TestMerge.java
Outdated
Show resolved
Hide resolved
| // | ||
| // UNPARTITIONED UNORDERED | ||
| // ------------------------------------------------------------------------- | ||
| // merge mode is NOT SET -> rely on write distribution and ordering as a basis |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I think by this you mean that we will check the write distribution mode and then use one of the following cases, right? So we can expect this to be unspecified distribution + LOCALLY ORDER BY _spec_id, _partition, _file, _pos because all 3 modes (whether merge or write mode) have that behavior?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Correct, I just did not want to duplicate all possible values of the write property. If the merge mode is not set, use the write mode and adapt it to cover deletes.
.../v3.2/spark/src/test/java/org/apache/iceberg/spark/TestSparkDistributionAndOrderingUtil.java
Show resolved
Hide resolved
| // merge mode is NOT SET -> rely on write distribution and ordering as a basis | ||
| // merge mode is NONE -> unspecified distribution + | ||
| // LOCALLY ORDER BY _spec_id, _partition, _file, _pos, id, data | ||
| // merge mode is HASH -> unspecified distribution + |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Why is the distribution unspecified when mode is hash here? Shouldn't this hash distribute by original data file (or maybe original partition), then by the new partition? Or is the fear that this will create too many small tasks by dividing original data files by the new partitioning?
If that's the case, then it seems like this is optimizing for the case where you're running a MERGE with data from an old partition spec. I'd rather optimize for the case where the partition spec matches.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Oh, I think I see. I was thinking about the PARTITIONED BY, UNORDERED case that is actually below. I concluded what you did for that case, so that's good validation!
Here, it still seems bad to me not to distribute. That's going to result in a lot of small delete files, which is really expensive and possibly worse than having a single writer for all the inserted data. It would be nice to be able to round-robin the new data... what about using something like HASH DISTRIBUTE BY _spec, _partition, bucket(id, data, numShufflePartitions)?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Well, I am not sure. I like that our merge and write logic are consistent right now. My hope was that AQE would coalesce tasks as needed to avoid a huge number of small writing tasks (and hence a huge number of delete files). I think AQE should behave better than a round-robin distribution. This case is about unpartitioned tables so we will most likely produce at a single delete file per writing task (that shouldn't be that bad). As long as we don't have a huge number of writing tasks, we should be fine, I guess?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
We chatted offline and decided that clustering by _spec_id, _partition, and _file is a good idea to avoid a large number of delete files.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Modified the behavior for HASH and RANGE modes.
rdblue
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Everything looks really close!
One question that I think I've answered, but I want to make sure about: when MERGE INTO gets rewritten to INSERT new rows because there are no MATCHED actions, we don't create a RowLevelOperationTable. That's the only place where we check the write mode and possibly create a delta operation and go into all the delta sort code path. Right?
I was briefly concerned that we might use a broken sort order for the INSERT cases, assuming that we create the sort order for deltas but actually rewrite to insert. But it looks like the write is always specific to the actual write operation.
|
@rdblue, you are right. There will be no |
szehon-ho
left a comment
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Looks good, just some very minor comments
|
|
||
| // use inner join if there is no NOT MATCHED action, unmatched source rows can be discarded | ||
| // use right outer join in all other cases, unmatched source rows may be needed | ||
| // also disable broadcasts for the target table to perform the cardinality check later |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Was also trying to compare the two code-paths. Noticed this has a 'later' in the comment and the other does not, the 'later' seems to be more clear. Actually wondering, did you consider putting some of these repeated codes/comments into single methods/scaladoc?
val sourceTableProj = sourceTableProj(source)
val joinPlan = joinPlan(sourceTableProj, targetTableProj, cond, notMatchedActions)
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Removed "later" for consistency in comments. The comments in copy-on-write and merge-on-read are slightly different as different join type are used. I'll take a look at what extra methods we can introduce to simplify this. I did that in a few places already.
...extensions/src/main/scala/org/apache/spark/sql/catalyst/analysis/RewriteMergeIntoTable.scala
Outdated
Show resolved
Hide resolved
| return deleteOrdering; | ||
| } else { | ||
| throw new IllegalArgumentException("Only position deletes and updates are currently supported"); | ||
| // all metadata columns like _spec_id, _file, _pos will be null for new data records |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Optional: just noticed we throw IllegalArgumentException in other places where we have an unknown command, so was just wondering would it be clearer/more consistent to put this in a MERGE case block and throw exception in unknown command case? Up to you
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It looks like we usually do that in switch blocks as it's required by the syntax. I think it should be safe enough to assume we know a finite set of commands.
|
Thanks for reviewing, @rdblue @szehon-ho! |
|
Awesome work, @aokolnychyi! |
This PR implements merge-on-read MERGE in Spark 3.2.