-
Notifications
You must be signed in to change notification settings - Fork 3.2k
Spark MERGE INTO Support (copy-on-write implementation) #1947
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
Changes from all commits
9db80c8
d82afba
35f6813
9cb2e86
227a108
9f264b7
6f29335
d7caece
9fadc1d
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,110 @@ | ||
| /* | ||
| * 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.catalyst.optimizer | ||
|
|
||
| import org.apache.spark.sql.catalyst.analysis.Resolver | ||
| import org.apache.spark.sql.catalyst.expressions.Alias | ||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.catalyst.expressions.InputFileName | ||
| import org.apache.spark.sql.catalyst.expressions.IsNull | ||
| import org.apache.spark.sql.catalyst.expressions.Literal | ||
| import org.apache.spark.sql.catalyst.plans.FullOuter | ||
| import org.apache.spark.sql.catalyst.plans.Inner | ||
| import org.apache.spark.sql.catalyst.plans.logical.DeleteAction | ||
| import org.apache.spark.sql.catalyst.plans.logical.InsertAction | ||
| import org.apache.spark.sql.catalyst.plans.logical.Join | ||
| import org.apache.spark.sql.catalyst.plans.logical.JoinHint | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.plans.logical.MergeAction | ||
| import org.apache.spark.sql.catalyst.plans.logical.MergeInto | ||
| import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams | ||
| import org.apache.spark.sql.catalyst.plans.logical.MergeIntoTable | ||
| import org.apache.spark.sql.catalyst.plans.logical.Project | ||
| import org.apache.spark.sql.catalyst.plans.logical.ReplaceData | ||
| import org.apache.spark.sql.catalyst.plans.logical.UpdateAction | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
| import org.apache.spark.sql.catalyst.utils.RewriteRowLevelOperationHelper | ||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation | ||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.types.BooleanType | ||
|
|
||
| case class RewriteMergeInto(conf: SQLConf) extends Rule[LogicalPlan] with RewriteRowLevelOperationHelper { | ||
| private val ROW_FROM_SOURCE = "_row_from_source_" | ||
| private val ROW_FROM_TARGET = "_row_from_target_" | ||
| private val TRUE_LITERAL = Literal(true, BooleanType) | ||
| private val FALSE_LITERAL = Literal(false, BooleanType) | ||
|
|
||
| import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._ | ||
|
|
||
| override def resolver: Resolver = conf.resolver | ||
|
|
||
| override def apply(plan: LogicalPlan): LogicalPlan = { | ||
| plan resolveOperators { | ||
| case MergeIntoTable(target: DataSourceV2Relation, source: LogicalPlan, cond, matchedActions, notMatchedActions) => | ||
| // Construct the plan to prune target based on join condition between source and target. | ||
| val writeInfo = newWriteInfo(target.schema) | ||
| val mergeBuilder = target.table.asMergeable.newMergeBuilder("merge", writeInfo) | ||
| val matchingRowsPlanBuilder = (rel: DataSourceV2ScanRelation) => | ||
| Join(source, rel, Inner, Some(cond), JoinHint.NONE) | ||
| val targetTableScan = buildScanPlan(target.table, target.output, mergeBuilder, cond, matchingRowsPlanBuilder) | ||
|
|
||
| // Construct an outer join to help track changes in source and target. | ||
| // TODO : Optimize this to use LEFT ANTI or RIGHT OUTER when applicable. | ||
| val sourceTableProj = source.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_SOURCE)()) | ||
| val targetTableProj = target.output ++ Seq(Alias(TRUE_LITERAL, ROW_FROM_TARGET)()) | ||
| val newTargetTableScan = Project(targetTableProj, targetTableScan) | ||
| val newSourceTableScan = Project(sourceTableProj, source) | ||
| val joinPlan = Join(newSourceTableScan, newTargetTableScan, FullOuter, Some(cond), JoinHint.NONE) | ||
|
|
||
| // Construct the plan to replace the data based on the output of `MergeInto` | ||
| val mergeParams = MergeIntoParams( | ||
| isSourceRowNotPresent = IsNull(findOutputAttr(joinPlan.output, ROW_FROM_SOURCE)), | ||
| isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan.output, ROW_FROM_TARGET)), | ||
| matchedConditions = matchedActions.map(getClauseCondition), | ||
| matchedOutputs = matchedActions.map(actionOutput(_, target.output)), | ||
| notMatchedConditions = notMatchedActions.map(getClauseCondition), | ||
| notMatchedOutputs = notMatchedActions.map(actionOutput(_, target.output)), | ||
| targetOutput = target.output :+ FALSE_LITERAL, | ||
| deleteOutput = target.output :+ TRUE_LITERAL, | ||
| joinedAttributes = joinPlan.output | ||
| ) | ||
| val mergePlan = MergeInto(mergeParams, target, joinPlan) | ||
| val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch() | ||
| ReplaceData(target, batchWrite, mergePlan) | ||
| } | ||
| } | ||
|
|
||
| private def actionOutput(clause: MergeAction, targetOutputCols: Seq[Expression]): Seq[Expression] = { | ||
| clause match { | ||
| case u: UpdateAction => | ||
| u.assignments.map(_.value) :+ FALSE_LITERAL | ||
| case _: DeleteAction => | ||
| targetOutputCols :+ TRUE_LITERAL | ||
| case i: InsertAction => | ||
| i.assignments.map(_.value) :+ FALSE_LITERAL | ||
| } | ||
| } | ||
|
|
||
| private def getClauseCondition(clause: MergeAction): Expression = { | ||
| clause.condition.getOrElse(TRUE_LITERAL) | ||
| } | ||
| } | ||
|
|
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,42 @@ | ||
| /* | ||
| * 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.catalyst.plans.logical | ||
|
|
||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation | ||
|
|
||
| case class MergeInto( | ||
| mergeIntoProcessor: MergeIntoParams, | ||
| targetRelation: DataSourceV2Relation, | ||
| child: LogicalPlan) extends UnaryNode { | ||
| override def output: Seq[Attribute] = targetRelation.output | ||
| } | ||
|
|
||
| case class MergeIntoParams( | ||
| isSourceRowNotPresent: Expression, | ||
| isTargetRowNotPresent: Expression, | ||
| matchedConditions: Seq[Expression], | ||
| matchedOutputs: Seq[Seq[Expression]], | ||
| notMatchedConditions: Seq[Expression], | ||
| notMatchedOutputs: Seq[Seq[Expression]], | ||
| targetOutput: Seq[Expression], | ||
| deleteOutput: Seq[Expression], | ||
| joinedAttributes: Seq[Attribute]) extends Serializable |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,135 @@ | ||
| /* | ||
| * 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.execution.datasources.v2 | ||
|
|
||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.sql.catalyst.InternalRow | ||
| import org.apache.spark.sql.catalyst.expressions.Attribute | ||
| import org.apache.spark.sql.catalyst.expressions.BasePredicate | ||
| import org.apache.spark.sql.catalyst.expressions.Expression | ||
| import org.apache.spark.sql.catalyst.expressions.UnsafeProjection | ||
| import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate | ||
| import org.apache.spark.sql.catalyst.plans.logical.MergeIntoParams | ||
| import org.apache.spark.sql.execution.SparkPlan | ||
| import org.apache.spark.sql.execution.UnaryExecNode | ||
|
|
||
| case class MergeIntoExec( | ||
| mergeIntoParams: MergeIntoParams, | ||
| @transient targetRelation: DataSourceV2Relation, | ||
| override val child: SparkPlan) extends UnaryExecNode { | ||
|
|
||
| override def output: Seq[Attribute] = targetRelation.output | ||
|
|
||
| protected override def doExecute(): RDD[InternalRow] = { | ||
| child.execute().mapPartitions { | ||
| processPartition(mergeIntoParams, _) | ||
| } | ||
| } | ||
|
|
||
| private def generateProjection(exprs: Seq[Expression], attrs: Seq[Attribute]): UnsafeProjection = { | ||
| UnsafeProjection.create(exprs, attrs) | ||
| } | ||
|
|
||
| private def generatePredicate(expr: Expression, attrs: Seq[Attribute]): BasePredicate = { | ||
| GeneratePredicate.generate(expr, attrs) | ||
| } | ||
|
|
||
| def applyProjection( | ||
| actions: Seq[(BasePredicate, UnsafeProjection)], | ||
| projectTargetCols: UnsafeProjection, | ||
| projectDeleteRow: UnsafeProjection, | ||
| inputRow: InternalRow, | ||
| targetRowNotPresent: Boolean): InternalRow = { | ||
|
|
||
|
|
||
| // Find the first combination where the predicate evaluates to true. | ||
| // In case when there are overlapping condition in the MATCHED | ||
| // clauses, for the first one that satisfies the predicate, the | ||
| // corresponding action is applied. For example: | ||
| // WHEN MATCHED AND id > 1 AND id < 10 UPDATE * | ||
| // WHEN MATCHED AND id = 5 OR id = 21 DELETE | ||
| // In above case, when id = 5, it applies both that matched predicates. In this | ||
| // case the first one we see is applied. | ||
|
|
||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Nit: no need for an empty comment and an empty line. |
||
| val pair = actions.find { | ||
| case (predicate, _) => predicate.eval(inputRow) | ||
| } | ||
|
rdblue marked this conversation as resolved.
|
||
|
|
||
| // Now apply the appropriate projection to either : | ||
| // - Insert a row into target | ||
| // - Update a row of target | ||
| // - Delete a row in target. The projected row will have the deleted bit set. | ||
| pair match { | ||
| case Some((_, projection)) => | ||
| projection.apply(inputRow) | ||
| case None => | ||
| if (targetRowNotPresent) { | ||
| projectDeleteRow.apply(inputRow) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. It seems a bit odd to apply this projection because the target row will be deleted. It seems like we could use the same lazily-initialized row for every delete.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @rdblue I had thought about it. But couldn't think of a way to do it. How about, we create a materialized delete row once per partition like this : deteExpr will come from rewriteMergeInto just like its passed now. Here we will just create the InternalRow once and use it ? Will that work Ryan ?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What if we just returned I think the problem is that this is trying to create one output for each input row, then filtering happens afterward. An extra column is added to signal that the row should be kept or not. But we don't need to copy the row if it is going to be removed. We also don't need to copy incoming target rows just to add a So what if we changed all of the delete cases to produce Let's not worry about this for now, but I'll open a PR after this is merged to simplify and avoid some of the copies.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @rdblue OK.
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @rdblue One thing to note is that, the output of the outer join is target cols + source cols. So we have to project out the necessary target columns, i think. |
||
| } else { | ||
| projectTargetCols.apply(inputRow) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| def processPartition( | ||
| params: MergeIntoParams, | ||
| rowIterator: Iterator[InternalRow]): Iterator[InternalRow] = { | ||
|
|
||
| val joinedAttrs = params.joinedAttributes | ||
| val isSourceRowNotPresentPred = generatePredicate(params.isSourceRowNotPresent, joinedAttrs) | ||
| val isTargetRowNotPresentPred = generatePredicate(params.isTargetRowNotPresent, joinedAttrs) | ||
| val matchedPreds = params.matchedConditions.map(generatePredicate(_, joinedAttrs)) | ||
| val matchedProjs = params.matchedOutputs.map(generateProjection(_, joinedAttrs)) | ||
| val notMatchedPreds = params.notMatchedConditions.map(generatePredicate(_, joinedAttrs)) | ||
| val notMatchedProjs = params.notMatchedOutputs.map(generateProjection(_, joinedAttrs)) | ||
| val projectTargetCols = generateProjection(params.targetOutput, joinedAttrs) | ||
| val projectDeletedRow = generateProjection(params.deleteOutput, joinedAttrs) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. These last two projections are only needed when In the rewrite, there is also a function, I think we could simplify the logic here and avoid extra clauses by ensuring that both
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I guess
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
Given this background, can you please explain your idea a little bit ?
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Let's clear this up in a follow-up. |
||
| val nonMatchedPairs = notMatchedPreds zip notMatchedProjs | ||
| val matchedPairs = matchedPreds zip matchedProjs | ||
|
|
||
| def shouldDeleteRow(row: InternalRow): Boolean = | ||
| row.getBoolean(params.targetOutput.size - 1) | ||
|
|
||
|
rdblue marked this conversation as resolved.
|
||
| /** | ||
| * This method is responsible for processing a input row to emit the resultant row with an | ||
| * additional column that indicates whether the row is going to be included in the final | ||
| * output of merge or not. | ||
| * 1. If there is a target row for which there is no corresponding source row (join condition not met) | ||
| * - Only project the target columns with deleted flag set to false. | ||
| * 2. If there is a source row for which there is no corresponding target row (join condition not met) | ||
| * - Apply the not matched actions (i.e INSERT actions) if non match conditions are met. | ||
| * 3. If there is a source row for which there is a corresponding target row (join condition met) | ||
| * - Apply the matched actions (i.e DELETE or UPDATE actions) if match conditions are met. | ||
| */ | ||
| def processRow(inputRow: InternalRow): InternalRow = { | ||
|
rdblue marked this conversation as resolved.
|
||
| if (isSourceRowNotPresentPred.eval(inputRow)) { | ||
| projectTargetCols.apply(inputRow) | ||
| } else if (isTargetRowNotPresentPred.eval(inputRow)) { | ||
| applyProjection(nonMatchedPairs, projectTargetCols, projectDeletedRow, inputRow, true) | ||
| } else { | ||
| applyProjection(matchedPairs, projectTargetCols, projectDeletedRow, inputRow, false) | ||
| } | ||
| } | ||
|
|
||
| rowIterator | ||
| .map(processRow) | ||
| .filterNot(shouldDeleteRow) | ||
| } | ||
| } | ||
Uh oh!
There was an error while loading. Please reload this page.