rdblue commented on a change in pull request #2022: URL: https://github.com/apache/iceberg/pull/2022#discussion_r560619673
########## File path: spark3-extensions/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteMergeInto.scala ########## @@ -0,0 +1,204 @@ +/* + * 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.iceberg.TableProperties.{MERGE_WRITE_CARDINALITY_CHECK, MERGE_WRITE_CARDINALITY_CHECK_DEFAULT, MERGE_WRITE_SORT_MODE, MERGE_WRITE_SORT_MODE_GLOBAL} +import org.apache.iceberg.spark.Spark3Util +import org.apache.iceberg.spark.Spark3Util.toRequiredDistribution +import org.apache.iceberg.spark.source.SparkTable +import org.apache.iceberg.util.PropertyUtil +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner} +import org.apache.spark.sql.catalyst.plans.logical.{MergeAction, _} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.utils.PlanHelper +import org.apache.spark.sql.connector.catalog.Table +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf + +case class RewriteMergeInto(spark: SparkSession) extends Rule[LogicalPlan] with PlanHelper with Logging { + val ROW_FROM_SOURCE = "_row_from_source_" + val ROW_FROM_TARGET = "_row_from_target_" + + import org.apache.spark.sql.execution.datasources.v2.ExtendedDataSourceV2Implicits._ + + override def apply(plan: LogicalPlan): LogicalPlan = { + plan resolveOperators { + // rewrite all operations that require reading the table to delete records + case MergeIntoTable(target: DataSourceV2Relation, + source: LogicalPlan, cond, actions, notActions) => + val targetOutputCols = target.output + val newProjectCols = target.output ++ + Seq( + Alias(InputFileName(), FILE_NAME_COL)(), + Alias(monotonically_increasing_id().expr, ROW_ID_COL)() + ) + val newTargetTable = Project(newProjectCols, target) + + // Construct the plan to prune target based on join condition between source and + // target. + val prunedTargetPlan = Join(source, newTargetTable, Inner, Some(cond), JoinHint.NONE) + val writeInfo = newWriteInfo(target.schema) + val mergeBuilder = target.table.asMergeable.newMergeBuilder("delete", writeInfo) + val targetTableScan = buildScanPlan(spark, target.table, + target.output, mergeBuilder, prunedTargetPlan, isCountCheckEnabled(target.table, actions)) + + // 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(lit(true).expr, ROW_FROM_SOURCE)()) + val targetTableProj = target.output ++ Seq(Alias(lit(true).expr, 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, ROW_FROM_SOURCE)), + isTargetRowNotPresent = IsNull(findOutputAttr(joinPlan, ROW_FROM_TARGET)), + matchedConditions = actions.map(getClauseCondition), + matchedOutputs = actions.map(actionOutput(_, targetOutputCols)), + notMatchedConditions = notActions.map(getClauseCondition), + notMatchedOutputs = notActions.map(actionOutput(_, targetOutputCols)), + targetOutput = targetOutputCols :+ Literal(false), + deleteOutput = targetOutputCols :+ Literal(true), + joinedAttributes = joinPlan.output + ) + val joinKeysFromTarget = targetOutputCols.filter ( + attr => cond.references.exists(attr.semanticEquals(_)) + ) + val writePlan = buildWritePlan(joinPlan, Project(target.output, joinPlan), target.table, joinKeysFromTarget) + val mergePlan = MergeInto(mergeParams, target, writePlan) + val batchWrite = mergeBuilder.asWriteBuilder.buildForBatch() + ReplaceData(target, batchWrite, mergePlan) + } + } + + def buildWritePlan(childPlan: LogicalPlan, + planToResolveFrom: LogicalPlan, + table: Table, targetJoinAttrs: Seq[Attribute]): LogicalPlan = { + val iceTable = table.asInstanceOf[SparkTable].table() + val globalSortEnabled = isGlobalSortEnabled(table) + val partitionExpressions = toCatalyst(toRequiredDistribution(iceTable.spec), planToResolveFrom) + val sortExpressions: Seq[SortOrder] = toCatalyst(toRequiredDistribution(iceTable.spec(), + iceTable.sortOrder(), false), planToResolveFrom).asInstanceOf[Seq[SortOrder]] + val dist = toRequiredDistribution(iceTable.spec(), + iceTable.sortOrder(), true) + val globalSortExprs: Seq[SortOrder] = toCatalyst(dist, planToResolveFrom).asInstanceOf[Seq[SortOrder]] Review comment: Following up on this, I think we should implement the proposal from our [discussion about Flink hash distribution](https://github.com/apache/iceberg/pull/2064#discussion_r559018257). That PR is going to add `write.distribution-mode` with 3 values: `none`, `hash`, and `range`. Here's what we would use them for here: | Spark | `none` | `hash` | `range` | |-|--------|-------------|--------| | unordered | no distribution, locally sort by partition key | hash distribute by partition key, locally sort by partition key | range distribute by partition key, locally sort by partition key | | ordered | no distribution, locally sorted | hash distribute by partition key, locally sorted | globally sorted | Or in terms of the methods you've added: | Spark | `none` | `hash` | `range` | |-|--------|-------------|--------| | unordered | no distribution, local sort by `toOrderedDistribution(spec, order, true)` | distribute by `toClusteredDistribution(spec)`, local sort by `toOrderedDistribution(spec, order, true)` | global sort by `toOrderedDistribution(spec, order, true)` | | ordered | no distribution, local sort by `toOrderedDistribution(spec, order, true)` | distribute by `toClusteredDistribution(spec)`, local sort by `toOrderedDistribution(spec, order, true)` | global sort by `toOrderedDistribution(spec, order, true)` | The result of `toOrderedDistribution(spec, order, true)` is always used to sort, whether locally or globally. If the sort order is unordered, then it will infer the order from the partition spec just as we wanted from the top table. For hash partitioning, we can use `toClusteredDistribution` that you added. Then the only other concern is that we need to add a round-robin shuffle before the global sort. With that addition, let me revise my code from above: ```scala val distributionMode = table.properties.getOrDefault("write.distribution-mode", "global") val order = toCatalyst(toOrderedDistribution(spec, order, true)) distributionMode.toLower(Locale.ROOT) match { case "none" => Sort(order, global = false, childPlan) case "hash" => val clustering = toCatalyst(toClusteredDistribution(spec)) val hashPartitioned = RepartitionByExpression(clustering, childPlan, numShufflePartitions) Sort(order, global = false, hashPartitioned) case "range" => val roundRobin = Repartition(numShufflePartitions, shuffle = true, childPlan) Sort(order, global = true, roundRobin) } ``` ---------------------------------------------------------------- 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. For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
