diff --git a/.github/workflows/utitcase-spark-4.x.yml b/.github/workflows/utitcase-spark-4.x.yml index 56629110f503..de0877fc185d 100644 --- a/.github/workflows/utitcase-spark-4.x.yml +++ b/.github/workflows/utitcase-spark-4.x.yml @@ -61,10 +61,10 @@ jobs: jvm_timezone=$(random_timezone) echo "JVM timezone is set to $jvm_timezone" test_modules="" - for suffix in ut 4.0; do + for suffix in ut 4.0 4.1; do test_modules+="org.apache.paimon:paimon-spark-${suffix}_2.13," done test_modules="${test_modules%,}" - mvn -T 2C -B -ntp verify -pl "${test_modules}" -Duser.timezone=$jvm_timezone -Pspark4,flink1 + mvn -B -ntp verify -pl "${test_modules}" -Duser.timezone=$jvm_timezone -Pspark4,flink1 env: MAVEN_OPTS: -Xmx4096m \ No newline at end of file diff --git a/paimon-spark/paimon-spark-4.1/pom.xml b/paimon-spark/paimon-spark-4.1/pom.xml new file mode 100644 index 000000000000..fa1da4c71a13 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/pom.xml @@ -0,0 +1,203 @@ + + + + 4.0.0 + + + org.apache.paimon + paimon-spark + 1.5-SNAPSHOT + + + paimon-spark-4.1_2.13 + Paimon : Spark : 4.1 : 2.13 + + + 4.1.1 + + 2.0.17 + + + + + org.apache.paimon + paimon-format + + + + org.apache.paimon + paimon-spark4-common_${scala.binary.version} + ${project.version} + + + + org.apache.paimon + paimon-spark-common_${scala.binary.version} + ${project.version} + + + + org.apache.spark + spark-sql-api_${scala.binary.version} + ${spark.version} + + + org.apache.spark + spark-connect-shims_${scala.binary.version} + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + + + + org.apache.spark + spark-hive_${scala.binary.version} + ${spark.version} + + + + + org.antlr + antlr4-runtime + ${antlr4.version} + + + + + org.apache.logging.log4j + log4j-slf4j2-impl + ${log4j.version} + test + + + + + + org.apache.paimon + paimon-spark-ut_${scala.binary.version} + ${project.version} + tests + test + + + * + * + + + + + + org.apache.spark + spark-sql_${scala.binary.version} + ${spark.version} + tests + test + + + org.apache.spark + spark-connect-shims_${scala.binary.version} + + + + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${spark.version} + tests + test + + + + org.apache.spark + spark-core_${scala.binary.version} + ${spark.version} + tests + test + + + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + true + + + + org.apache.maven.plugins + maven-shade-plugin + + + shade-paimon + package + + shade + + + + + * + + com/github/luben/zstd/** + **/*libzstd-jni-*.so + **/*libzstd-jni-*.dll + + + + + + org.apache.paimon:paimon-spark4-common_${scala.binary.version} + + + + + + + + + diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/SparkTable.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/SparkTable.scala new file mode 100644 index 000000000000..a96ad66d5ef2 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/SparkTable.scala @@ -0,0 +1,36 @@ +/* + * 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.paimon.spark + +import org.apache.paimon.table.Table + +/** + * Spark 4.1 shim for SparkTable. + * + * In Spark 4.1, RewriteMergeIntoTable / RewriteDeleteFromTable / RewriteUpdateTable were moved into + * the Resolution batch, running BEFORE Paimon's post-hoc resolution rules. If SparkTable implements + * SupportsRowLevelOperations, Spark's built-in rewrite rules match and rewrite MergeIntoTable / + * DeleteFromTable / UpdateTable using the V2 write path, which Paimon's PK/DV tables do not + * support. + * + * This shim removes SupportsRowLevelOperations so that MergeIntoTable.rewritable returns false, + * preventing Spark's rewrite rules from matching. Paimon's post-hoc rules (PaimonMergeInto, + * PaimonDeleteTable, PaimonUpdateTable) then handle these commands correctly. + */ +case class SparkTable(override val table: Table) extends PaimonSparkTableBase(table) diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/AssignmentAlignmentHelper.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/AssignmentAlignmentHelper.scala new file mode 100644 index 000000000000..c0f7af90439c --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/AssignmentAlignmentHelper.scala @@ -0,0 +1,180 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.spark.SparkTypeUtils.CURRENT_DEFAULT_COLUMN_METADATA_KEY +import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateNamedStruct, Expression, GetStructField, Literal, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.{Assignment, DeleteAction, InsertAction, InsertStarAction, MergeAction, MergeIntoTable, UpdateAction, UpdateStarAction} +import org.apache.spark.sql.types.StructType + +trait AssignmentAlignmentHelper extends SQLConfHelper with ExpressionHelper { + + private lazy val resolver = conf.resolver + + /** + * @param ref + * attribute reference seq, e.g. a => Seq["a"], s.c1 => Seq["s", "c1"] + * @param expr + * update expression + */ + private case class AttrUpdate(ref: Seq[String], expr: Expression) + + /** + * Generate aligned expressions, only supports PrimitiveType and StructType. For example, if attrs + * are [a int, b int, s struct(c1 int, c2 int)] and update assignments are [a = 1, s.c1 = 2], will + * return [1, b, struct(2, c2)]. + * @param attrs + * target attrs + * @param assignments + * update assignments + * @return + * aligned expressions + */ + protected def generateAlignedExpressions( + attrs: Seq[Attribute], + assignments: Seq[Assignment], + isInsert: Boolean = false): Seq[Expression] = { + val attrUpdates = assignments.map(a => AttrUpdate(toRefSeq(a.key), a.value)) + recursiveAlignUpdates(attrs, attrUpdates, Nil, isInsert) + } + + protected def alignAssignments( + attrs: Seq[Attribute], + assignments: Seq[Assignment], + isInsert: Boolean = false): Seq[Assignment] = { + generateAlignedExpressions(attrs, assignments, isInsert).zip(attrs).map { + case (expression, field) => Assignment(field, expression) + } + } + + /** + * Align assignments in a MergeAction based on the target table's output attributes. + * - DeleteAction: returns as-is + * - UpdateAction: aligns assignments for update + * - InsertAction: aligns assignments for insert + */ + protected def alignMergeAction(action: MergeAction, targetOutput: Seq[Attribute]): MergeAction = { + action match { + case d @ DeleteAction(_) => d + case u @ UpdateAction(_, assignments, _) => + u.copy(assignments = alignAssignments(targetOutput, assignments)) + case i @ InsertAction(_, assignments) => + i.copy(assignments = alignAssignments(targetOutput, assignments, isInsert = true)) + case _: UpdateStarAction => + throw new RuntimeException("UpdateStarAction should not be here.") + case _: InsertStarAction => + throw new RuntimeException("InsertStarAction should not be here.") + case _ => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + private def recursiveAlignUpdates( + targetAttrs: Seq[NamedExpression], + updates: Seq[AttrUpdate], + namePrefix: Seq[String] = Nil, + isInsert: Boolean = false): Seq[Expression] = { + + // build aligned updated expression for each target attr + targetAttrs.map { + targetAttr => + val headMatchedUpdates = updates.filter(u => resolver(u.ref.head, targetAttr.name)) + if (headMatchedUpdates.isEmpty) { + if (isInsert) { + // For Insert, use default value or NULL for missing columns + getDefaultValueOrNull(targetAttr) + } else { + // For Update, return the attr as is + targetAttr + } + } else { + val exactMatchedUpdate = headMatchedUpdates.find(_.ref.size == 1) + if (exactMatchedUpdate.isDefined) { + if (headMatchedUpdates.size == 1) { + // when an exact match (no nested fields) occurs, it must be the only match, then return it's expr + castIfNeeded(exactMatchedUpdate.get.expr, targetAttr.dataType) + } else { + // otherwise, there must be conflicting updates, for example: + // - update the same attr multiple times + // - update a struct attr and its fields at the same time (e.g. s and s.c1) + val conflictingAttrNames = + headMatchedUpdates.map(u => (namePrefix ++ u.ref).mkString(".")).distinct + throw new UnsupportedOperationException( + s"Conflicting update/insert on attrs: ${conflictingAttrNames.mkString(", ")}" + ) + } + } else { + targetAttr.dataType match { + case StructType(fields) => + val fieldExprs = fields.zipWithIndex.map { + case (field, ordinal) => + Alias(GetStructField(targetAttr, ordinal, Some(field.name)), field.name)() + } + val newUpdates = updates.map(u => u.copy(ref = u.ref.tail)) + // process StructType's nested fields recursively + val updatedFieldExprs = + recursiveAlignUpdates( + fieldExprs, + newUpdates, + namePrefix :+ targetAttr.name, + isInsert) + + // build updated struct expression + CreateNamedStruct(fields.zip(updatedFieldExprs).flatMap { + case (field, expr) => + Seq(Literal(field.name), expr) + }) + case _ => + // can't reach here + throw new UnsupportedOperationException("") + } + } + } + } + } + + /** Get the default value expression for an attribute, or NULL if no default value is defined. */ + private def getDefaultValueOrNull(attr: NamedExpression): Expression = { + attr match { + case a: Attribute if a.metadata.contains(CURRENT_DEFAULT_COLUMN_METADATA_KEY) => + val defaultValueStr = a.metadata.getString(CURRENT_DEFAULT_COLUMN_METADATA_KEY) + parseAndResolveDefaultValue(defaultValueStr, a) + case _ => + Literal(null, attr.dataType) + } + } + + /** Parse the default value string and resolve it to an expression. */ + private def parseAndResolveDefaultValue(defaultValueStr: String, attr: Attribute): Expression = { + try { + val spark = SparkSession.active + val parsed = spark.sessionState.sqlParser.parseExpression(defaultValueStr) + castIfNeeded(parsed, attr.dataType) + } catch { + case _: Exception => + // If parsing fails, fall back to NULL + Literal(null, attr.dataType) + } + } + +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala new file mode 100644 index 000000000000..74bfd239ff75 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolver.scala @@ -0,0 +1,59 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical.{DeleteAction, LogicalPlan, MergeAction, MergeIntoTable, UpdateAction} + +/** Resolve all the expressions for MergeInto. */ +object PaimonMergeIntoResolver extends PaimonMergeIntoResolverBase { + + def resolveNotMatchedBySourceActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + merge.notMatchedBySourceActions.map { + case DeleteAction(condition) => + // The condition must be from the target table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, TARGET_ONLY)) + DeleteAction(resolvedCond) + case UpdateAction(condition, assignments, _) => + // The condition and value must be from the target table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, TARGET_ONLY)) + val resolvedAssignments = resolveAssignments(resolve, assignments, merge, TARGET_ONLY) + UpdateAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + def build( + merge: MergeIntoTable, + resolvedCond: Expression, + resolvedMatched: Seq[MergeAction], + resolvedNotMatched: Seq[MergeAction], + resolvedNotMatchedBySource: Seq[MergeAction]): MergeIntoTable = { + merge.copy( + mergeCondition = resolvedCond, + matchedActions = resolvedMatched, + notMatchedActions = resolvedNotMatched, + notMatchedBySourceActions = resolvedNotMatchedBySource + ) + } + +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala new file mode 100644 index 000000000000..596f9970d26b --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonMergeIntoResolverBase.scala @@ -0,0 +1,143 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.plans.logical._ + +trait PaimonMergeIntoResolverBase extends ExpressionHelper { + + def apply(merge: MergeIntoTable, spark: SparkSession): LogicalPlan = { + val target = merge.targetTable + val source = merge.sourceTable + assert(target.resolved, "Target should have been resolved here.") + assert(source.resolved, "Source should have been resolved here.") + + val resolve: (Expression, LogicalPlan) => Expression = resolveExpression(spark) + + val resolvedCond = resolveCondition(resolve, merge.mergeCondition, merge, ALL) + val resolvedMatched = resolveMatchedByTargetActions(merge, resolve) + val resolvedNotMatched = resolveNotMatchedByTargetActions(merge, resolve) + val resolvedNotMatchedBySource = resolveNotMatchedBySourceActions(merge, resolve) + + build(merge, resolvedCond, resolvedMatched, resolvedNotMatched, resolvedNotMatchedBySource) + } + + def build( + merge: MergeIntoTable, + resolvedCond: Expression, + resolvedMatched: Seq[MergeAction], + resolvedNotMatched: Seq[MergeAction], + resolvedNotMatchedBySource: Seq[MergeAction]): MergeIntoTable + + private def resolveMatchedByTargetActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + merge.matchedActions.map { + case DeleteAction(condition) => + // The condition can be from both target and source tables + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + DeleteAction(resolvedCond) + case UpdateAction(condition, assignments, _) => + // The condition and value can be from both target and source tables + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + val resolvedAssignments = resolveAssignments(resolve, assignments, merge, ALL) + UpdateAction(resolvedCond, resolvedAssignments) + case UpdateStarAction(condition) => + // The condition can be from both target and source tables, but the value must be from the source table + val resolvedCond = condition.map(resolveCondition(resolve, _, merge, ALL)) + val assignments = merge.targetTable.output.map { + attr => Assignment(attr, UnresolvedAttribute(Seq(attr.name))) + } + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + UpdateAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + private def resolveNotMatchedByTargetActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] = { + merge.notMatchedActions.map { + case InsertAction(condition, assignments) => + // The condition and value must be from the source table + val resolvedCond = + condition.map(resolveCondition(resolve, _, merge, SOURCE_ONLY)) + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + InsertAction(resolvedCond, resolvedAssignments) + case InsertStarAction(condition) => + // The condition and value must be from the source table + val resolvedCond = + condition.map(resolveCondition(resolve, _, merge, SOURCE_ONLY)) + val assignments = merge.targetTable.output.map { + attr => Assignment(attr, UnresolvedAttribute(Seq(attr.name))) + } + val resolvedAssignments = + resolveAssignments(resolve, assignments, merge, SOURCE_ONLY) + InsertAction(resolvedCond, resolvedAssignments) + case action => + throw new RuntimeException(s"Can't recognize this action: $action") + } + } + + def resolveNotMatchedBySourceActions( + merge: MergeIntoTable, + resolve: (Expression, LogicalPlan) => Expression): Seq[MergeAction] + + sealed trait ResolvedWith + case object ALL extends ResolvedWith + case object SOURCE_ONLY extends ResolvedWith + case object TARGET_ONLY extends ResolvedWith + + def resolveCondition( + resolve: (Expression, LogicalPlan) => Expression, + condition: Expression, + mergeInto: MergeIntoTable, + resolvedWith: ResolvedWith): Expression = { + resolvedWith match { + case ALL => resolve(condition, mergeInto) + case SOURCE_ONLY => resolve(condition, Project(Nil, mergeInto.sourceTable)) + case TARGET_ONLY => resolve(condition, Project(Nil, mergeInto.targetTable)) + } + } + + def resolveAssignments( + resolve: (Expression, LogicalPlan) => Expression, + assignments: Seq[Assignment], + mergeInto: MergeIntoTable, + resolvedWith: ResolvedWith): Seq[Assignment] = { + assignments.map { + assign => + val resolvedKey = resolve(assign.key, Project(Nil, mergeInto.targetTable)) + val resolvedValue = resolvedWith match { + case ALL => resolve(assign.value, mergeInto) + case SOURCE_ONLY => resolve(assign.value, Project(Nil, mergeInto.sourceTable)) + case TARGET_ONLY => resolve(assign.value, Project(Nil, mergeInto.targetTable)) + } + Assignment(resolvedKey, resolvedValue) + } + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonRelation.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonRelation.scala new file mode 100644 index 000000000000..2eb69521c51d --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonRelation.scala @@ -0,0 +1,58 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.spark.SparkTable + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, ResolvedTable} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation + +import scala.util.control.NonFatal + +/** An analysis helper */ +object PaimonRelation extends Logging { + + def unapply(plan: LogicalPlan): Option[SparkTable] = + EliminateSubqueryAliases(plan) match { + case Project(_, DataSourceV2Relation(table: SparkTable, _, _, _, _, _)) => Some(table) + case DataSourceV2Relation(table: SparkTable, _, _, _, _, _) => Some(table) + case ResolvedTable(_, _, table: SparkTable, _) => Some(table) + case _ => None + } + + def isPaimonTable(plan: LogicalPlan): Boolean = { + try { + PaimonRelation.unapply(plan).nonEmpty + } catch { + case NonFatal(e) => + logWarning("Can't check if this plan is a paimon table", e) + false + } + } + + def getPaimonRelation(plan: LogicalPlan): DataSourceV2Relation = { + EliminateSubqueryAliases(plan) match { + case Project(_, d @ DataSourceV2Relation(_: SparkTable, _, _, _, _, _)) => d + case d @ DataSourceV2Relation(_: SparkTable, _, _, _, _, _) => d + case _ => throw new RuntimeException(s"It's not a paimon table, $plan") + } + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala new file mode 100644 index 000000000000..5bb9c2b83bb2 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/PaimonViewResolver.scala @@ -0,0 +1,113 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.catalog.Catalog.ViewNotExistException +import org.apache.paimon.spark.SparkTypeUtils +import org.apache.paimon.spark.catalog.SupportView +import org.apache.paimon.view.View + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{CTESubstitution, GetColumnByOrdinal, UnresolvedRelation, UnresolvedTableOrView} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, UpCast} +import org.apache.spark.sql.catalyst.parser.ParseException +import org.apache.spark.sql.catalyst.parser.extensions.{CurrentOrigin, Origin} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Project, SubqueryAlias} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.util.CharVarcharUtils +import org.apache.spark.sql.connector.catalog.{Identifier, PaimonLookupCatalog} + +/** + * Spark 4.1 shim for PaimonViewResolver. + * + * In Spark 4.1, SubstituteUnresolvedOrdinals was removed and its functionality was refactored into + * the resolver package. This shim removes the reference to SubstituteUnresolvedOrdinals from the + * earlyRules sequence. Ordinal substitution is handled by the Analyzer's Resolution batch in Spark + * 4.1. + */ +case class PaimonViewResolver(spark: SparkSession) + extends Rule[LogicalPlan] + with PaimonLookupCatalog { + + protected lazy val catalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { + case u @ UnresolvedRelation(parts @ CatalogAndIdentifier(catalog: SupportView, ident), _, _) => + try { + val view = catalog.loadView(ident) + createViewRelation(parts, view) + } catch { + case _: ViewNotExistException => + u + } + + case u @ UnresolvedTableOrView(CatalogAndIdentifier(catalog: SupportView, ident), _, _) => + try { + catalog.loadView(ident) + ResolvedPaimonView(catalog, ident) + } catch { + case _: ViewNotExistException => + u + } + } + + private def createViewRelation(nameParts: Seq[String], view: View): LogicalPlan = { + val parsedPlan = + parseViewText(nameParts.toArray.mkString("."), view.query(SupportView.DIALECT)) + + // Apply early analysis rules that won't re-run for plans injected during Resolution batch. + // SubstituteUnresolvedOrdinals was removed in Spark 4.1; ordinal substitution is now + // handled within the Analyzer's Resolution batch. + val earlyRules = Seq(CTESubstitution) + val rewritten = earlyRules.foldLeft(parsedPlan)((plan, rule) => rule.apply(plan)) + + val viewSchema = CharVarcharUtils.replaceCharVarcharWithStringInSchema( + SparkTypeUtils.fromPaimonRowType(view.rowType())) + + val aliases = viewSchema.fields.zipWithIndex.map { + case (expected, pos) => + val attr = GetColumnByOrdinal(pos, expected.dataType) + Alias(UpCast(attr, expected.dataType), expected.name)(explicitMetadata = + Some(expected.metadata)) + } + + SubqueryAlias(nameParts, Project(aliases, rewritten)) + } + + private def parseViewText(name: String, viewText: String): LogicalPlan = { + val origin = Origin( + objectType = Some("VIEW"), + objectName = Some(name) + ) + try { + CurrentOrigin.withOrigin(origin) { + try { + spark.sessionState.sqlParser.parseQuery(viewText) + } catch { + // For compatibility with Spark 3.2 and below + case _: NoSuchMethodError => + spark.sessionState.sqlParser.parsePlan(viewText) + } + } + } catch { + case _: ParseException => + throw new RuntimeException("Failed to parse view text: " + viewText) + } + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/RewriteUpsertTable.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/RewriteUpsertTable.scala new file mode 100644 index 000000000000..fd8776d6dbb4 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/analysis/RewriteUpsertTable.scala @@ -0,0 +1,129 @@ +/* + * 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.paimon.spark.catalyst.analysis + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.table.FileStoreTable + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} +import org.apache.spark.sql.catalyst.expressions.{Alias, And, Attribute, CurrentRow, Descending, EqualNullSafe, EqualTo, LessThanOrEqual, Literal, RowFrame, RowNumber, SortOrder, SpecifiedWindowFrame, UnboundedPreceding, WindowExpression, WindowSpecDefinition} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.paimon.shims.SparkShimLoader + +import scala.collection.JavaConverters._ + +/** Rewrite upsert table to merge into. */ +case class RewriteUpsertTable(spark: SparkSession) extends Rule[LogicalPlan] { + + override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsDown { + case p @ PaimonV2WriteCommand(table) => + val (usingUpsert, upsertKey, sequenceField) = usingUpsertTable(table) + if (!usingUpsert) { + return p + } + + p match { + case AppendData(target, source, _, _, _, _) => + val deduplicatedSource = if (sequenceField.nonEmpty) { + deduplicateBySequenceField(source, upsertKey, sequenceField) + } else { + source + } + + rewriteToMergeInto(target, deduplicatedSource, upsertKey, sequenceField) + case _ => p + } + } + + private def usingUpsertTable(table: DataSourceV2Relation): (Boolean, Seq[String], Seq[String]) = { + table.table match { + case SparkTable(fileStoreTable: FileStoreTable) => + val coreOptions = fileStoreTable.coreOptions() + val upsertKey = coreOptions.upsertKey().asScala.toSeq + val sequenceField = coreOptions.sequenceField().asScala.toSeq + if (fileStoreTable.primaryKeys().isEmpty && upsertKey.nonEmpty) { + (true, upsertKey, sequenceField) + } else { + (false, Seq.empty, Seq.empty) + } + case _ => (false, Seq.empty, Seq.empty) + } + } + + private def deduplicateBySequenceField( + source: LogicalPlan, + upsertKey: Seq[String], + sequenceField: Seq[String]): LogicalPlan = { + val winSpec = WindowSpecDefinition( + cols(source.output, upsertKey), + cols(source.output, sequenceField).map(SortOrder(_, Descending)), + SpecifiedWindowFrame(RowFrame, UnboundedPreceding, CurrentRow) + ) + val rnAlias = Alias(WindowExpression(RowNumber(), winSpec), "__rn__")() + val withRN = Project(UnresolvedStar(None) :: rnAlias :: Nil, source) + val filtered = Filter(EqualTo(UnresolvedAttribute("__rn__"), Literal(1)), withRN) + Project(source.output, filtered) + } + + private def rewriteToMergeInto( + target: LogicalPlan, + source: LogicalPlan, + upsertKey: Seq[String], + sequenceField: Seq[String] + ): MergeIntoTable = { + val mergeCondition = upsertKey + .map(k => EqualNullSafe(col(target.output, k), col(source.output, k))) + .reduce(And) + + val updateCondiction = if (sequenceField.nonEmpty) { + Option.apply( + sequenceField + .map(s => LessThanOrEqual(col(target.output, s), col(source.output, s))) + .reduce(And)) + } else { + Option.empty + } + + val assignments: Seq[Assignment] = + target.output.zip(source.output).map(a => Assignment(a._1, a._2)) + + val mergeActions = Seq(UpdateAction(updateCondiction, assignments)) + val notMatchedActions = Seq(InsertAction(None, assignments)) + + SparkShimLoader.shim.createMergeIntoTable( + target, + source, + mergeCondition, + mergeActions, + notMatchedActions, + Seq.empty, + withSchemaEvolution = false) + } + + private def cols(input: Seq[Attribute], colsNames: Seq[String]): Seq[Attribute] = { + colsNames.map(c => col(input, c)) + } + + private def col(input: Seq[Attribute], colsName: String): Attribute = { + input.find(_.name == colsName).get + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala new file mode 100644 index 000000000000..e86195f1af0b --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueries.scala @@ -0,0 +1,92 @@ +/* + * 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.paimon.spark.catalyst.optimizer + +import org.apache.paimon.spark.PaimonScan + +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, ExprId, ScalarSubquery, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation + +object MergePaimonScalarSubqueries extends MergePaimonScalarSubqueriesBase { + + override def tryMergeDataSourceV2ScanRelation( + newV2ScanRelation: DataSourceV2ScanRelation, + cachedV2ScanRelation: DataSourceV2ScanRelation) + : Option[(LogicalPlan, AttributeMap[Attribute])] = { + (newV2ScanRelation, cachedV2ScanRelation) match { + case ( + DataSourceV2ScanRelation( + newRelation, + newScan: PaimonScan, + newOutput, + newPartitioning, + newOrdering), + DataSourceV2ScanRelation( + cachedRelation, + cachedScan: PaimonScan, + _, + cachedPartitioning, + cacheOrdering)) => + checkIdenticalPlans(newRelation, cachedRelation).flatMap { + outputMap => + if ( + samePartitioning(newPartitioning, cachedPartitioning, outputMap) && sameOrdering( + newOrdering, + cacheOrdering, + outputMap) + ) { + mergePaimonScan(newScan, cachedScan).map { + mergedScan => + val mergedAttributes = mergedScan + .readSchema() + .map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) + val cachedOutputNameMap = cachedRelation.output.map(a => a.name -> a).toMap + val mergedOutput = + mergedAttributes.map(a => cachedOutputNameMap.getOrElse(a.name, a)) + val newV2ScanRelation = + cachedV2ScanRelation.copy(scan = mergedScan, output = mergedOutput) + + val mergedOutputNameMap = mergedOutput.map(a => a.name -> a).toMap + val newOutputMap = + AttributeMap(newOutput.map(a => a -> mergedOutputNameMap(a.name).toAttribute)) + + newV2ScanRelation -> newOutputMap + } + } else { + None + } + } + + case _ => None + } + } + + private def sameOrdering( + newOrdering: Option[Seq[SortOrder]], + cachedOrdering: Option[Seq[SortOrder]], + outputAttrMap: AttributeMap[Attribute]): Boolean = { + val mappedNewOrdering = newOrdering.map(_.map(mapAttributes(_, outputAttrMap))) + mappedNewOrdering.map(_.map(_.canonicalized)) == cachedOrdering.map(_.map(_.canonicalized)) + } + + override protected def createScalarSubquery(plan: LogicalPlan, exprId: ExprId): ScalarSubquery = { + ScalarSubquery(plan, exprId = exprId) + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala new file mode 100644 index 000000000000..d6507a6baf85 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/MergePaimonScalarSubqueriesBase.scala @@ -0,0 +1,349 @@ +/* + * 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.paimon.spark.catalyst.optimizer + +import org.apache.paimon.spark.PaimonScan + +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, CreateNamedStruct, Expression, ExprId, GetStructField, LeafExpression, Literal, NamedExpression, PredicateHelper, ScalarSubquery, Unevaluable} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CTERelationDef, Filter, Join, LogicalPlan, Project, Subquery, WithCTE} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY, SCALAR_SUBQUERY_REFERENCE, TreePattern} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanRelation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.paimon.shims.SparkShimLoader +import org.apache.spark.sql.types.{DataType, StructType} + +import scala.collection.mutable.ArrayBuffer + +/** + * Spark 4.1 shim for MergePaimonScalarSubqueriesBase. + * + * In Spark 4.1, CTERelationDef gained a 5th parameter (maxDepth). The base trait in + * paimon-spark-common was compiled against Spark 4.0.2's 4-parameter CTERelationDef, causing + * NoSuchMethodError at runtime. This shim recompiles the entire trait against Spark 4.1.1. + */ +trait MergePaimonScalarSubqueriesBase extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = { + plan match { + // Subquery reuse needs to be enabled for this optimization. + case _ if !conf.getConf(SQLConf.SUBQUERY_REUSE_ENABLED) && !existsPaimonScan(plan) => plan + + // This rule does a whole plan traversal, no need to run on subqueries. + case _: Subquery => plan + + // Plans with CTEs are not supported for now. + case _: WithCTE => plan + + case _ => extractCommonScalarSubqueries(plan) + } + } + + private def existsPaimonScan(plan: LogicalPlan): Boolean = { + plan.find { + case r: DataSourceV2ScanRelation => r.scan.isInstanceOf[PaimonScan] + case _ => false + }.isDefined + } + + case class Header(attributes: Seq[Attribute], plan: LogicalPlan, merged: Boolean) + + private def extractCommonScalarSubqueries(plan: LogicalPlan) = { + val cache = ArrayBuffer.empty[Header] + val planWithReferences = insertReferences(plan, cache) + cache.zipWithIndex.foreach { + case (header, i) => + cache(i) = cache(i).copy(plan = if (header.merged) { + CTERelationDef( + createProject(header.attributes, removeReferences(header.plan, cache)), + underSubquery = true) + } else { + removeReferences(header.plan, cache) + }) + } + val newPlan = removeReferences(planWithReferences, cache) + val subqueryCTEs = cache.filter(_.merged).map(_.plan.asInstanceOf[CTERelationDef]) + if (subqueryCTEs.nonEmpty) { + WithCTE(newPlan, subqueryCTEs.toSeq) + } else { + newPlan + } + } + + // First traversal builds up the cache and inserts `ScalarSubqueryReference`s to the plan. + private def insertReferences(plan: LogicalPlan, cache: ArrayBuffer[Header]): LogicalPlan = { + plan.transformUpWithSubqueries { + case n => + n.transformExpressionsUpWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY)) { + case s: ScalarSubquery if !s.isCorrelated && s.deterministic => + val (subqueryIndex, headerIndex) = cacheSubquery(s.plan, cache) + ScalarSubqueryReference(subqueryIndex, headerIndex, s.dataType, s.exprId) + } + } + } + + private def cacheSubquery(plan: LogicalPlan, cache: ArrayBuffer[Header]): (Int, Int) = { + val output = plan.output.head + cache.zipWithIndex + .collectFirst(Function.unlift { + case (header, subqueryIndex) => + checkIdenticalPlans(plan, header.plan) + .map { + outputMap => + val mappedOutput = mapAttributes(output, outputMap) + val headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) + subqueryIndex -> headerIndex + } + .orElse(tryMergePlans(plan, header.plan).map { + case (mergedPlan, outputMap) => + val mappedOutput = mapAttributes(output, outputMap) + var headerIndex = header.attributes.indexWhere(_.exprId == mappedOutput.exprId) + val newHeaderAttributes = if (headerIndex == -1) { + headerIndex = header.attributes.size + header.attributes :+ mappedOutput + } else { + header.attributes + } + cache(subqueryIndex) = Header(newHeaderAttributes, mergedPlan, merged = true) + subqueryIndex -> headerIndex + }) + }) + .getOrElse { + cache += Header(Seq(output), plan, merged = false) + cache.length - 1 -> 0 + } + } + + protected def checkIdenticalPlans( + newPlan: LogicalPlan, + cachedPlan: LogicalPlan): Option[AttributeMap[Attribute]] = { + if (newPlan.canonicalized == cachedPlan.canonicalized) { + Some(AttributeMap(newPlan.output.zip(cachedPlan.output))) + } else { + None + } + } + + private def tryMergePlans( + newPlan: LogicalPlan, + cachedPlan: LogicalPlan): Option[(LogicalPlan, AttributeMap[Attribute])] = { + checkIdenticalPlans(newPlan, cachedPlan) + .map(cachedPlan -> _) + .orElse((newPlan, cachedPlan) match { + case (np: Project, cp: Project) => + tryMergePlans(np.child, cp.child).map { + case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.projectList, outputMap, cp.projectList) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np, cp: Project) => + tryMergePlans(np, cp.child).map { + case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.output, outputMap, cp.projectList) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np: Project, cp) => + tryMergePlans(np.child, cp).map { + case (mergedChild, outputMap) => + val (mergedProjectList, newOutputMap) = + mergeNamedExpressions(np.projectList, outputMap, cp.output) + val mergedPlan = Project(mergedProjectList, mergedChild) + mergedPlan -> newOutputMap + } + case (np: Aggregate, cp: Aggregate) if supportedAggregateMerge(np, cp) => + tryMergePlans(np.child, cp.child).flatMap { + case (mergedChild, outputMap) => + val mappedNewGroupingExpression = + np.groupingExpressions.map(mapAttributes(_, outputMap)) + if ( + mappedNewGroupingExpression.map(_.canonicalized) == + cp.groupingExpressions.map(_.canonicalized) + ) { + val (mergedAggregateExpressions, newOutputMap) = + mergeNamedExpressions(np.aggregateExpressions, outputMap, cp.aggregateExpressions) + val mergedPlan = + Aggregate(cp.groupingExpressions, mergedAggregateExpressions, mergedChild) + Some(mergedPlan -> newOutputMap) + } else { + None + } + } + + case (np: Filter, cp: Filter) => + tryMergePlans(np.child, cp.child).flatMap { + case (mergedChild, outputMap) => + val mappedNewCondition = mapAttributes(np.condition, outputMap) + if (mappedNewCondition.canonicalized == cp.condition.canonicalized) { + val mergedPlan = cp.withNewChildren(Seq(mergedChild)) + Some(mergedPlan -> outputMap) + } else { + None + } + } + + case (np: Join, cp: Join) if np.joinType == cp.joinType && np.hint == cp.hint => + tryMergePlans(np.left, cp.left).flatMap { + case (mergedLeft, leftOutputMap) => + tryMergePlans(np.right, cp.right).flatMap { + case (mergedRight, rightOutputMap) => + val outputMap = leftOutputMap ++ rightOutputMap + val mappedNewCondition = np.condition.map(mapAttributes(_, outputMap)) + if ( + mappedNewCondition.map(_.canonicalized) == cp.condition.map(_.canonicalized) + ) { + val mergedPlan = cp.withNewChildren(Seq(mergedLeft, mergedRight)) + Some(mergedPlan -> outputMap) + } else { + None + } + } + } + case ( + newV2ScanRelation: DataSourceV2ScanRelation, + cachedV2ScanRelation: DataSourceV2ScanRelation) => + tryMergeDataSourceV2ScanRelation(newV2ScanRelation, cachedV2ScanRelation) + + case _ => None + }) + } + + def tryMergeDataSourceV2ScanRelation( + newV2ScanRelation: DataSourceV2ScanRelation, + cachedV2ScanRelation: DataSourceV2ScanRelation) + : Option[(LogicalPlan, AttributeMap[Attribute])] + + protected def samePartitioning( + newPartitioning: Option[Seq[Expression]], + cachedPartitioning: Option[Seq[Expression]], + outputAttrMap: AttributeMap[Attribute]): Boolean = { + val mappedNewPartitioning = newPartitioning.map(_.map(mapAttributes(_, outputAttrMap))) + mappedNewPartitioning.map(_.map(_.canonicalized)) == cachedPartitioning.map( + _.map(_.canonicalized)) + } + + protected def mergePaimonScan(scan1: PaimonScan, scan2: PaimonScan): Option[PaimonScan] = { + if (scan1 == scan2) { + Some(scan2) + } else if (scan1 == scan2.copy(requiredSchema = scan1.requiredSchema)) { + val mergedRequiredSchema = StructType( + (scan2.requiredSchema.fields.toSet ++ scan1.requiredSchema.fields.toSet).toArray) + Some(scan2.copy(requiredSchema = mergedRequiredSchema)) + } else { + None + } + } + + private def createProject(attributes: Seq[Attribute], plan: LogicalPlan): Project = { + Project( + Seq( + Alias( + CreateNamedStruct(attributes.flatMap(a => Seq(Literal(a.name), a))), + "mergedValue")()), + plan) + } + + protected def mapAttributes[T <: Expression](expr: T, outputMap: AttributeMap[Attribute]): T = { + expr.transform { case a: Attribute => outputMap.getOrElse(a, a) }.asInstanceOf[T] + } + + private def mergeNamedExpressions( + newExpressions: Seq[NamedExpression], + outputMap: AttributeMap[Attribute], + cachedExpressions: Seq[NamedExpression]) = { + val mergedExpressions = ArrayBuffer[NamedExpression](cachedExpressions: _*) + val newOutputMap = AttributeMap(newExpressions.map { + ne => + val mapped = mapAttributes(ne, outputMap) + val withoutAlias = mapped match { + case Alias(child, _) => child + case e => e + } + ne.toAttribute -> mergedExpressions + .find { + case Alias(child, _) => child.semanticEquals(withoutAlias) + case e => e.semanticEquals(withoutAlias) + } + .getOrElse { + mergedExpressions += mapped + mapped + } + .toAttribute + }) + (mergedExpressions.toSeq, newOutputMap) + } + + private def supportedAggregateMerge(newPlan: Aggregate, cachedPlan: Aggregate): Boolean = { + val aggregateExpressionsSeq = Seq(newPlan, cachedPlan).map { + plan => plan.aggregateExpressions.flatMap(_.collect { case a: AggregateExpression => a }) + } + val groupByExpressionSeq = Seq(newPlan, cachedPlan).map(_.groupingExpressions) + + val Seq(newPlanSupportsHashAggregate, cachedPlanSupportsHashAggregate) = + aggregateExpressionsSeq.zip(groupByExpressionSeq).map { + case (aggregateExpressions, groupByExpressions) => + SparkShimLoader.shim.supportsHashAggregate( + aggregateExpressions.flatMap(_.aggregateFunction.aggBufferAttributes), + groupByExpressions) + } + + newPlanSupportsHashAggregate && cachedPlanSupportsHashAggregate || + newPlanSupportsHashAggregate == cachedPlanSupportsHashAggregate && { + val Seq(newPlanSupportsObjectHashAggregate, cachedPlanSupportsObjectHashAggregate) = + aggregateExpressionsSeq.zip(groupByExpressionSeq).map { + case (aggregateExpressions, groupByExpressions: Seq[Expression]) => + SparkShimLoader.shim + .supportsObjectHashAggregate(aggregateExpressions, groupByExpressions) + } + newPlanSupportsObjectHashAggregate && cachedPlanSupportsObjectHashAggregate || + newPlanSupportsObjectHashAggregate == cachedPlanSupportsObjectHashAggregate + } + } + + private def removeReferences(plan: LogicalPlan, cache: ArrayBuffer[Header]) = { + plan.transformUpWithSubqueries { + case n => + n.transformExpressionsWithPruning(_.containsAnyPattern(SCALAR_SUBQUERY_REFERENCE)) { + case ssr: ScalarSubqueryReference => + val header = cache(ssr.subqueryIndex) + if (header.merged) { + val subqueryCTE = header.plan.asInstanceOf[CTERelationDef] + GetStructField( + createScalarSubquery( + SparkShimLoader.shim.createCTERelationRef( + subqueryCTE.id, + resolved = true, + subqueryCTE.output, + isStreaming = subqueryCTE.isStreaming), + ssr.exprId), + ssr.headerIndex + ) + } else { + createScalarSubquery(header.plan, ssr.exprId) + } + } + } + } + + protected def createScalarSubquery(plan: LogicalPlan, exprId: ExprId): ScalarSubquery + +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/ScalarSubqueryReference.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/ScalarSubqueryReference.scala new file mode 100644 index 000000000000..7d3f0bcaf944 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/optimizer/ScalarSubqueryReference.scala @@ -0,0 +1,46 @@ +/* + * 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.paimon.spark.catalyst.optimizer + +import org.apache.spark.sql.catalyst.expressions.{ExprId, LeafExpression, Unevaluable} +import org.apache.spark.sql.catalyst.trees.TreePattern.{SCALAR_SUBQUERY_REFERENCE, TreePattern} +import org.apache.spark.sql.types.DataType + +/** + * Spark 4.1 shim for ScalarSubqueryReference. + * + * In Spark 4.0.2, Unevaluable extends FoldableUnevaluable. In Spark 4.1.1, FoldableUnevaluable was + * removed. The base class in paimon-spark-common was compiled against 4.0.2 and its bytecode + * references FoldableUnevaluable, causing ClassNotFoundException at runtime. This shim redefines + * the class against Spark 4.1.1's Unevaluable. + */ +case class ScalarSubqueryReference( + subqueryIndex: Int, + headerIndex: Int, + dataType: DataType, + exprId: ExprId) + extends LeafExpression + with Unevaluable { + override def nullable: Boolean = true + + final override val nodePatterns: Seq[TreePattern] = Seq(SCALAR_SUBQUERY_REFERENCE) + + override def stringArgs: Iterator[Any] = + Iterator(subqueryIndex, headerIndex, dataType, exprId.id) +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala new file mode 100644 index 000000000000..647222bbdd6a --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/catalyst/plans/logical/PaimonTableValuedFunctions.scala @@ -0,0 +1,214 @@ +/* + * 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.paimon.spark.catalyst.plans.logical + +import org.apache.paimon.CoreOptions +import org.apache.paimon.predicate.{FullTextSearch, VectorSearch} +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.catalyst.plans.logical.PaimonTableValuedFunctions._ +import org.apache.paimon.table.{DataTable, FullTextSearchTable, InnerTable, VectorSearchTable} +import org.apache.paimon.table.source.snapshot.TimeTravelUtil.InconsistentTagBucketException + +import org.apache.spark.sql.PaimonUtils.createDataset +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.FunctionRegistryBase +import org.apache.spark.sql.catalyst.analysis.TableFunctionRegistry.TableFunctionBuilder +import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateArray, Expression, ExpressionInfo, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.connector.catalog.{Identifier, Table, TableCatalog} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import scala.collection.JavaConverters._ + +object PaimonTableValuedFunctions { + + val INCREMENTAL_QUERY = "paimon_incremental_query" + val INCREMENTAL_BETWEEN_TIMESTAMP = "paimon_incremental_between_timestamp" + val INCREMENTAL_TO_AUTO_TAG = "paimon_incremental_to_auto_tag" + val VECTOR_SEARCH = "vector_search" + val FULL_TEXT_SEARCH = "full_text_search" + + val supportedFnNames: Seq[String] = + Seq( + INCREMENTAL_QUERY, + INCREMENTAL_BETWEEN_TIMESTAMP, + INCREMENTAL_TO_AUTO_TAG, + VECTOR_SEARCH, + FULL_TEXT_SEARCH) + + private type TableFunctionDescription = (FunctionIdentifier, ExpressionInfo, TableFunctionBuilder) + + def getTableValueFunctionInjection(fnName: String): TableFunctionDescription = { + val (info, builder) = fnName match { + case INCREMENTAL_QUERY => + FunctionRegistryBase.build[IncrementalQuery](fnName, since = None) + case INCREMENTAL_BETWEEN_TIMESTAMP => + FunctionRegistryBase.build[IncrementalBetweenTimestamp](fnName, since = None) + case INCREMENTAL_TO_AUTO_TAG => + FunctionRegistryBase.build[IncrementalToAutoTag](fnName, since = None) + case VECTOR_SEARCH => + FunctionRegistryBase.build[VectorSearchQuery](fnName, since = None) + case FULL_TEXT_SEARCH => + FunctionRegistryBase.build[FullTextSearchQuery](fnName, since = None) + case _ => + throw new Exception(s"Function $fnName isn't a supported table valued function.") + } + val ident = FunctionIdentifier(fnName) + (ident, info, builder) + } + + def resolvePaimonTableValuedFunction( + spark: SparkSession, + tvf: PaimonTableValueFunction): LogicalPlan = { + val args = tvf.expressions + + val sessionState = spark.sessionState + val catalogManager = sessionState.catalogManager + + val identifier = args.head.eval().toString + val (catalogName, dbName, tableName) = { + sessionState.sqlParser.parseMultipartIdentifier(identifier) match { + case Seq(table) => + (catalogManager.currentCatalog.name(), catalogManager.currentNamespace.head, table) + case Seq(db, table) => (catalogManager.currentCatalog.name(), db, table) + case Seq(catalog, db, table) => (catalog, db, table) + case _ => throw new RuntimeException(s"Invalid table identifier: $identifier") + } + } + + val sparkCatalog = catalogManager.catalog(catalogName).asInstanceOf[TableCatalog] + val ident: Identifier = Identifier.of(Array(dbName), tableName) + val sparkTable = sparkCatalog.loadTable(ident) + + // Handle vector_search and full_text_search specially + tvf match { + case vsq: VectorSearchQuery => + resolveVectorSearchQuery(sparkTable, sparkCatalog, ident, vsq, args.tail) + case ftsq: FullTextSearchQuery => + resolveFullTextSearchQuery(sparkTable, sparkCatalog, ident, ftsq, args.tail) + case _ => + val options = tvf.parseArgs(args.tail) + usingSparkIncrementQuery(tvf, sparkTable, options) match { + case Some(snapshotIdPair: (Long, Long)) => + sparkIncrementQuery(spark, sparkTable, sparkCatalog, ident, options, snapshotIdPair) + case _ => + DataSourceV2Relation.create( + sparkTable, + Some(sparkCatalog), + Some(ident), + new CaseInsensitiveStringMap(options.asJava)) + } + } + } + + private def resolveVectorSearchQuery( + sparkTable: Table, + sparkCatalog: TableCatalog, + ident: Identifier, + vsq: VectorSearchQuery, + argsWithoutTable: Seq[Expression]): LogicalPlan = { + sparkTable match { + case st @ SparkTable(innerTable: InnerTable) => + val vectorSearch = vsq.createVectorSearch(innerTable, argsWithoutTable) + val vectorSearchTable = VectorSearchTable.create(innerTable, vectorSearch) + DataSourceV2Relation.create( + st.copy(table = vectorSearchTable), + Some(sparkCatalog), + Some(ident), + CaseInsensitiveStringMap.empty()) + case _ => + throw new RuntimeException( + "vector_search only supports Paimon SparkTable backed by InnerTable, " + + s"but got table implementation: ${sparkTable.getClass.getName}") + } + } + + private def resolveFullTextSearchQuery( + sparkTable: Table, + sparkCatalog: TableCatalog, + ident: Identifier, + ftsq: FullTextSearchQuery, + argsWithoutTable: Seq[Expression]): LogicalPlan = { + sparkTable match { + case st @ SparkTable(innerTable: InnerTable) => + val fullTextSearch = ftsq.createFullTextSearch(innerTable, argsWithoutTable) + val fullTextSearchTable = FullTextSearchTable.create(innerTable, fullTextSearch) + DataSourceV2Relation.create( + st.copy(table = fullTextSearchTable), + Some(sparkCatalog), + Some(ident), + CaseInsensitiveStringMap.empty()) + case _ => + throw new RuntimeException( + "full_text_search only supports Paimon SparkTable backed by InnerTable, " + + s"but got table implementation: ${sparkTable.getClass.getName}") + } + } + + private def usingSparkIncrementQuery( + tvf: PaimonTableValueFunction, + sparkTable: Table, + options: Map[String, String]): Option[(Long, Long)] = { + tvf.fnName match { + case INCREMENTAL_QUERY | INCREMENTAL_TO_AUTO_TAG => + sparkTable match { + case SparkTable(fileStoreTable: DataTable) => + try { + fileStoreTable.copy(options.asJava).asInstanceOf[DataTable].newScan().plan() + None + } catch { + case e: InconsistentTagBucketException => + Some((e.startSnapshotId, e.endSnapshotId)) + } + case _ => None + } + case _ => None + } + } + + private def sparkIncrementQuery( + spark: SparkSession, + sparkTable: Table, + sparkCatalog: TableCatalog, + ident: Identifier, + options: Map[String, String], + snapshotIdPair: (Long, Long)): LogicalPlan = { + val filteredOptions = + options - CoreOptions.INCREMENTAL_BETWEEN.key - CoreOptions.INCREMENTAL_TO_AUTO_TAG.key + + def datasetOfSnapshot(snapshotId: Long) = { + val updatedOptions = filteredOptions + (CoreOptions.SCAN_VERSION.key() -> snapshotId.toString) + createDataset( + spark, + DataSourceV2Relation.create( + sparkTable, + Some(sparkCatalog), + Some(ident), + new CaseInsensitiveStringMap(updatedOptions.asJava) + )) + } + + datasetOfSnapshot(snapshotIdPair._2) + .except(datasetOfSnapshot(snapshotIdPair._1)) + .queryExecution + .logical + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonDataEvolutionTable.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonDataEvolutionTable.scala new file mode 100644 index 000000000000..3a85870c7293 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonDataEvolutionTable.scala @@ -0,0 +1,616 @@ +/* + * 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.paimon.spark.commands + +import org.apache.paimon.CoreOptions.GlobalIndexColumnUpdateAction +import org.apache.paimon.data.BinaryRow +import org.apache.paimon.format.blob.BlobFileFormat.isBlobFile +import org.apache.paimon.io.{CompactIncrement, DataIncrement} +import org.apache.paimon.manifest.IndexManifestEntry +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.catalyst.analysis.PaimonRelation +import org.apache.paimon.spark.catalyst.analysis.PaimonRelation.isPaimonTable +import org.apache.paimon.spark.catalyst.analysis.PaimonUpdateTable.toColumn +import org.apache.paimon.spark.leafnode.PaimonLeafRunnableCommand +import org.apache.paimon.spark.util.ScanPlanHelper.createNewScanPlan +import org.apache.paimon.table.FileStoreTable +import org.apache.paimon.table.sink.{CommitMessage, CommitMessageImpl} +import org.apache.paimon.table.source.DataSplit + +import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.PaimonUtils._ +import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer.resolver +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, EqualTo, Expression, ExprId, Literal} +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} +import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftOuter} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.MergeRows.Keep +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.functions.{col, udf} +import org.apache.spark.sql.types.StructType + +import scala.collection.{immutable, mutable} +import scala.collection.JavaConverters._ +import scala.collection.Searching.{search, Found, InsertionPoint} +import scala.collection.mutable.{ArrayBuffer, ListBuffer} + +/** Command for Merge Into for Data Evolution paimon table. */ +case class MergeIntoPaimonDataEvolutionTable( + v2Table: SparkTable, + targetTable: LogicalPlan, + sourceTable: LogicalPlan, + matchedCondition: Expression, + matchedActions: Seq[MergeAction], + notMatchedActions: Seq[MergeAction], + notMatchedBySourceActions: Seq[MergeAction]) + extends PaimonLeafRunnableCommand + with WithFileStoreTable { + + private lazy val writer = PaimonSparkWriter(table) + + assert( + notMatchedBySourceActions.isEmpty, + "notMatchedBySourceActions is not supported in MergeIntoPaimonDataEvolutionTable.") + assert( + matchedActions.forall(x => x.isInstanceOf[UpdateAction]), + "Only SET clause is supported in MergeIntoPaimonDataEvolutionTable for SQL: WHEN MATCHED.") + assert( + notMatchedActions.forall(x => x.isInstanceOf[InsertAction]), + "Only INSERT clause is supported in MergeIntoPaimonDataEvolutionTable for SQL: WHEN NOT MATCHED." + ) + + import MergeIntoPaimonDataEvolutionTable._ + + override val table: FileStoreTable = v2Table.getTable.asInstanceOf[FileStoreTable] + + private val updateColumns: Set[AttributeReference] = { + val columns = mutable.Set[AttributeReference]() + for (action <- matchedActions) { + action match { + case updateAction: UpdateAction => + for (assignment <- updateAction.assignments) { + if (!assignment.key.equals(assignment.value)) { + val key = assignment.key.asInstanceOf[AttributeReference] + columns ++= Seq(key) + } + } + } + } + columns.toSet + } + + /** + * Self-Merge pattern: + * {{{ + * MERGE INTO T AS t + * USING T AS s + * ON t._ROW_ID = s._ROW_ID + * WHEN MATCHED THEN UPDATE ... SET ... + * }}} + * For this pattern, the execution can be optimized to: + * + * `Scan -> MergeRows -> Write` + * + * without any extra shuffle, join, or sort. + */ + private lazy val isSelfMergeOnRowId: Boolean = { + if (!isPaimonTable(sourceTable)) { + false + } else if (!targetRelation.name.equals(PaimonRelation.getPaimonRelation(sourceTable).name)) { + false + } else { + matchedCondition match { + case EqualTo(left: AttributeReference, right: AttributeReference) + if left.name == ROW_ID_NAME && right.name == ROW_ID_NAME => + true + case _ => false + } + } + } + + assert( + !(isSelfMergeOnRowId && (notMatchedActions.nonEmpty || notMatchedBySourceActions.nonEmpty)), + "Self-Merge on _ROW_ID only supports WHEN MATCHED THEN UPDATE. WHEN NOT MATCHED and WHEN " + + "NOT MATCHED BY SOURCE are not supported." + ) + + private lazy val targetRelation: DataSourceV2Relation = + PaimonRelation.getPaimonRelation(targetTable) + + lazy val tableSchema: StructType = v2Table.schema + + override def run(sparkSession: SparkSession): Seq[Row] = { + invokeMergeInto(sparkSession) + Seq.empty[Row] + } + + private def invokeMergeInto(sparkSession: SparkSession): Unit = { + val plan = table.newSnapshotReader().read() + val tableSplits: Seq[DataSplit] = plan + .splits() + .asScala + .map(_.asInstanceOf[DataSplit]) + .toSeq + + val firstRowIds: immutable.IndexedSeq[Long] = tableSplits + .flatMap(_.dataFiles().asScala) + .filter(file => file.firstRowId() != null && !isBlobFile(file.fileName())) + .map(file => file.firstRowId().asInstanceOf[Long]) + .distinct + .sorted + .toIndexedSeq + + val firstRowIdToBlobFirstRowIds: Map[Long, List[Long]] = { + val map = new mutable.HashMap[Long, List[Long]]() + val files = tableSplits + .flatMap(_.dataFiles().asScala) + .filter(file => isBlobFile(file.fileName())) + .sortBy(f => f.firstRowId()) + + for (file <- files) { + val firstRowId = file.firstRowId().asInstanceOf[Long] + val firstIdInNormalFile = floorBinarySearch(firstRowIds, firstRowId) + map.update( + firstIdInNormalFile, + map.getOrElseUpdate(firstIdInNormalFile, List.empty[Long]) :+ firstRowId + ) + } + map.toMap + } + + // step 1: find the related data splits, make it target file plan + val dataSplits: Seq[DataSplit] = + targetRelatedSplits(sparkSession, tableSplits, firstRowIds, firstRowIdToBlobFirstRowIds) + val touchedFileTargetRelation = + createNewScanPlan(dataSplits, targetRelation) + + // step 2: invoke update action + val updateCommit = + if (matchedActions.nonEmpty) { + val updateResult = + updateActionInvoke(dataSplits, sparkSession, touchedFileTargetRelation, firstRowIds) + checkUpdateResult(updateResult) + } else Nil + + // step 3: invoke insert action + val insertCommit = + if (notMatchedActions.nonEmpty) + insertActionInvoke(sparkSession, touchedFileTargetRelation) + else Nil + + if (plan.snapshotId() != null) { + writer.rowIdCheckConflict(plan.snapshotId()) + } + writer.commit(updateCommit ++ insertCommit) + } + + private def targetRelatedSplits( + sparkSession: SparkSession, + tableSplits: Seq[DataSplit], + firstRowIds: immutable.IndexedSeq[Long], + firstRowIdToBlobFirstRowIds: Map[Long, List[Long]]): Seq[DataSplit] = { + // Self-Merge shortcut: + // In Self-Merge mode, every row in the table may be updated, so we scan all splits. + if (isSelfMergeOnRowId) { + return tableSplits + } + + val sourceDss = createDataset(sparkSession, sourceTable) + + val firstRowIdsTouched = extractSourceRowIdMapping match { + case Some(sourceRowIdAttr) => + // Shortcut: Directly get _FIRST_ROW_IDs from the source table. + findRelatedFirstRowIds( + sourceDss, + sparkSession, + firstRowIds, + firstRowIdToBlobFirstRowIds, + sourceRowIdAttr.name).toSet + + case None => + // Perform the full join to find related _FIRST_ROW_IDs. + val targetDss = createDataset(sparkSession, targetRelation) + findRelatedFirstRowIds( + targetDss.alias("_left").join(sourceDss, toColumn(matchedCondition), "inner"), + sparkSession, + firstRowIds, + firstRowIdToBlobFirstRowIds, + "_left." + ROW_ID_NAME + ).toSet + } + + tableSplits + .map( + split => + split.filterDataFile( + file => file.firstRowId() != null && firstRowIdsTouched.contains(file.firstRowId()))) + .filter(optional => optional.isPresent) + .map(_.get()) + } + + private def updateActionInvoke( + dataSplits: Seq[DataSplit], + sparkSession: SparkSession, + touchedFileTargetRelation: DataSourceV2Relation, + firstRowIds: immutable.IndexedSeq[Long]): Seq[CommitMessage] = { + val mergeFields = extractFields(matchedCondition) + val allFields = mutable.SortedSet.empty[AttributeReference]( + (o1, o2) => { + o1.toString().compareTo(o2.toString()) + }) ++ mergeFields + + val updateColumnsSorted = updateColumns.toSeq.sortBy( + s => targetTable.output.map(x => x.toString()).indexOf(s.toString())) + + // Different Spark versions might produce duplicate attributes between `output` and + // `metadataOutput`, so manually deduplicate by `exprId`. + val metadataColumns = (targetRelation.output ++ targetRelation.metadataOutput) + .filter(attr => attr.name.equals(ROW_ID_NAME)) + .groupBy(_.exprId) + .map { case (_, attrs) => attrs.head } + .toSeq + + val assignments = metadataColumns.map(column => Assignment(column, column)) + val output = updateColumnsSorted ++ metadataColumns + val realUpdateActions = matchedActions + .map(s => s.asInstanceOf[UpdateAction]) + .map( + update => + UpdateAction( + update.condition, + update.assignments.filter( + a => + updateColumnsSorted.contains( + a.key.asInstanceOf[AttributeReference])) ++ assignments)) + + for (action <- realUpdateActions) { + allFields ++= action.references.flatMap(r => extractFields(r)).seq + } + + val toWrite = if (isSelfMergeOnRowId) { + // Self-Merge shortcut: + // - Scan the target table only (no source scan, no join), and read all columns required by + // merge condition and update expressions. + // - Rewrite all source-side AttributeReferences to the corresponding target attributes. + // - The scan output already satisfies the required partitioning and ordering for partial + // updates, so no extra shuffle or sort is needed. + + val targetAttrsDedup: Seq[AttributeReference] = + (targetRelation.output ++ targetRelation.metadataOutput) + .groupBy(_.exprId) + .map { case (_, attrs) => attrs.head } + .toSeq + + val neededNames: Set[String] = (allFields ++ metadataColumns).map(_.name).toSet + val allReadFieldsOnTarget: Seq[AttributeReference] = + targetAttrsDedup.filter(a => neededNames.exists(n => resolver(n, a.name))) + val readPlan = touchedFileTargetRelation.copy(output = allReadFieldsOnTarget) + + // Build mapping: source exprId -> target attr (matched by column name). + val sourceToTarget = { + val targetAttrs = targetRelation.output ++ targetRelation.metadataOutput + val sourceAttrs = sourceTable.output ++ sourceTable.metadataOutput + sourceAttrs.flatMap { + s => targetAttrs.find(t => resolver(t.name, s.name)).map(t => s.exprId -> t) + }.toMap + } + + def rewriteSourceToTarget( + expr: Expression, + m: Map[ExprId, AttributeReference]): Expression = { + expr.transform { + case a: AttributeReference if m.contains(a.exprId) => m(a.exprId) + } + } + + val rewrittenUpdateActions: Seq[UpdateAction] = realUpdateActions.map { + ua => + val newCond = ua.condition.map(c => rewriteSourceToTarget(c, sourceToTarget)) + val newAssignments = ua.assignments.map { + a => Assignment(a.key, rewriteSourceToTarget(a.value, sourceToTarget)) + } + ua.copy(condition = newCond, assignments = newAssignments) + } + + val mergeRows = MergeRows( + isSourceRowPresent = TrueLiteral, + isTargetRowPresent = TrueLiteral, + matchedInstructions = rewrittenUpdateActions + .map( + action => { + Keep( + MergeRows.Update, + action.condition.getOrElse(TrueLiteral), + action.assignments.map(a => a.value)) + }) ++ Seq(Keep(MergeRows.Copy, TrueLiteral, output)), + notMatchedInstructions = Nil, + notMatchedBySourceInstructions = Seq(Keep(MergeRows.Copy, TrueLiteral, output)), + checkCardinality = false, + output = output, + child = readPlan + ) + + val withFirstRowId = addFirstRowId(sparkSession, mergeRows, firstRowIds) + assert(withFirstRowId.schema.fields.length == updateColumnsSorted.size + 2) + withFirstRowId + } else { + val allReadFieldsOnTarget = allFields.filter( + field => + targetTable.output.exists(attr => attr.exprId.equals(field.exprId))) ++ metadataColumns + val allReadFieldsOnSource = + allFields.filter( + field => sourceTable.output.exists(attr => attr.exprId.equals(field.exprId))) + + val targetReadPlan = + touchedFileTargetRelation.copy(output = allReadFieldsOnTarget.toSeq) + val targetTableProjExprs = targetReadPlan.output :+ Alias(TrueLiteral, ROW_FROM_TARGET)() + val targetTableProj = Project(targetTableProjExprs, targetReadPlan) + + val sourceTableProjExprs = + allReadFieldsOnSource.toSeq :+ Alias(TrueLiteral, ROW_FROM_SOURCE)() + val sourceTableProj = Project(sourceTableProjExprs, sourceTable) + + val joinPlan = + Join(targetTableProj, sourceTableProj, LeftOuter, Some(matchedCondition), JoinHint.NONE) + val rowFromSourceAttr = attribute(ROW_FROM_SOURCE, joinPlan) + val rowFromTargetAttr = attribute(ROW_FROM_TARGET, joinPlan) + val mergeRows = MergeRows( + isSourceRowPresent = rowFromSourceAttr, + isTargetRowPresent = rowFromTargetAttr, + matchedInstructions = realUpdateActions + .map( + action => { + Keep( + MergeRows.Update, + action.condition.getOrElse(TrueLiteral), + action.assignments.map(a => a.value)) + }) ++ Seq(Keep(MergeRows.Copy, TrueLiteral, output)), + notMatchedInstructions = Nil, + notMatchedBySourceInstructions = Seq(Keep(MergeRows.Copy, TrueLiteral, output)).toSeq, + checkCardinality = false, + output = output, + child = joinPlan + ) + val withFirstRowId = addFirstRowId(sparkSession, mergeRows, firstRowIds) + assert(withFirstRowId.schema.fields.length == updateColumnsSorted.size + 2) + withFirstRowId + .repartition(col(FIRST_ROW_ID_NAME)) + .sortWithinPartitions(FIRST_ROW_ID_NAME, ROW_ID_NAME) + } + + val writer = DataEvolutionPaimonWriter(table, dataSplits) + writer.writePartialFields(toWrite, updateColumnsSorted.map(_.name)) + } + + private def insertActionInvoke( + sparkSession: SparkSession, + touchedFileTargetRelation: DataSourceV2Relation): Seq[CommitMessage] = { + val mergeFields = extractFields(matchedCondition) + val allReadFieldsOnTarget = + mergeFields.filter(field => targetTable.output.exists(attr => attr.equals(field))) + + val targetReadPlan = + touchedFileTargetRelation.copy(targetRelation.table, allReadFieldsOnTarget.toSeq) + + val joinPlan = + Join(sourceTable, targetReadPlan, LeftAnti, Some(matchedCondition), JoinHint.NONE) + + // merge rows as there are multiple not matched actions + val mergeRows = MergeRows( + isSourceRowPresent = TrueLiteral, + isTargetRowPresent = FalseLiteral, + matchedInstructions = Nil, + notMatchedInstructions = notMatchedActions.map { + case insertAction: InsertAction => + Keep( + MergeRows.Insert, + insertAction.condition.getOrElse(TrueLiteral), + insertAction.assignments.map( + a => + if ( + !a.value.isInstanceOf[AttributeReference] || joinPlan.output.exists( + attr => attr.toString().equals(a.value.toString())) + ) a.value + else Literal(null)) + ) + }.toSeq, + notMatchedBySourceInstructions = Nil, + checkCardinality = false, + output = targetTable.output, + child = joinPlan + ) + + val toWrite = createDataset(sparkSession, mergeRows) + writer.write(toWrite) + } + + /** + * Attempts to identify a direct mapping from sourceTable's attribute to the target table's + * `_ROW_ID`. + * + * This is a shortcut optimization for `MERGE INTO` to avoid a full, expensive join when the merge + * condition is a simple equality on the target's `_ROW_ID`. + * + * @return + * An `Option` containing the sourceTable's attribute if a pattern like + * `target._ROW_ID = source.col` (or its reverse) is found, otherwise `None`. + */ + private def extractSourceRowIdMapping: Option[AttributeReference] = { + + // Helper to check if an attribute is the target's _ROW_ID + def isTargetRowId(attr: AttributeReference): Boolean = { + attr.name == ROW_ID_NAME && (targetRelation.output ++ targetRelation.metadataOutput) + .exists(_.exprId.equals(attr.exprId)) + } + + // Helper to check if an attribute belongs to the source table + def isSourceAttribute(attr: AttributeReference): Boolean = { + (sourceTable.output ++ sourceTable.metadataOutput).exists(_.exprId.equals(attr.exprId)) + } + + matchedCondition match { + // Case 1: target._ROW_ID = source.col + case EqualTo(left: AttributeReference, right: AttributeReference) + if isTargetRowId(left) && isSourceAttribute(right) => + Some(right) + // Case 2: source.col = target._ROW_ID + case EqualTo(left: AttributeReference, right: AttributeReference) + if isSourceAttribute(left) && isTargetRowId(right) => + Some(left) + case _ => None + } + } + + private def checkUpdateResult(updateCommit: Seq[CommitMessage]): Seq[CommitMessage] = { + val affectedParts: Set[BinaryRow] = updateCommit.map(_.partition()).toSet + val rowType = table.rowType() + + // find all global index files of affected partitions and updated columns + val latestSnapshot = table.latestSnapshot() + if (!latestSnapshot.isPresent) { + return updateCommit + } + + val filter: org.apache.paimon.utils.Filter[IndexManifestEntry] = + (entry: IndexManifestEntry) => { + val globalIndexMeta = entry.indexFile().globalIndexMeta() + if (globalIndexMeta == null) { + false + } else { + val fieldName = rowType.getField(globalIndexMeta.indexFieldId()).name() + affectedParts.contains(entry.partition()) && updateColumns.exists( + _.name.equals(fieldName)) + } + } + + val affectedIndexEntries = table + .store() + .newIndexFileHandler() + .scan(latestSnapshot.get(), filter) + .asScala + + if (affectedIndexEntries.isEmpty) { + updateCommit + } else { + table.coreOptions().globalIndexColumnUpdateAction() match { + case GlobalIndexColumnUpdateAction.THROW_ERROR => + val updatedColNames = updateColumns.map(_.name) + val conflicted = affectedIndexEntries + .map(_.indexFile().globalIndexMeta().indexFieldId()) + .map(id => rowType.getField(id).name()) + .toSet + throw new RuntimeException( + s"""MergeInto: update columns contain globally indexed columns, not supported now. + |Updated columns: ${updatedColNames.toSeq.sorted.mkString("[", ", ", "]")} + |Conflicted columns: ${conflicted.toSeq.sorted.mkString("[", ", ", "]")} + |""".stripMargin) + case GlobalIndexColumnUpdateAction.DROP_PARTITION_INDEX => + val grouped = affectedIndexEntries.groupBy(_.partition()) + val deleteCommitMessages = ArrayBuffer.empty[CommitMessage] + grouped.foreach { + case (part, entries) => + deleteCommitMessages += new CommitMessageImpl( + part, + 0, + null, + DataIncrement.deleteIndexIncrement(entries.map(_.indexFile()).asJava), + CompactIncrement.emptyIncrement()) + } + updateCommit ++ deleteCommitMessages + } + } + } + + private def findRelatedFirstRowIds( + dataset: Dataset[Row], + sparkSession: SparkSession, + firstRowIds: immutable.IndexedSeq[Long], + firstRowIdToBlobFirstRowIds: Map[Long, List[Long]], + identifier: String): Array[Long] = { + import sparkSession.implicits._ + val firstRowIdUdf = udf((rowId: Long) => floorBinarySearch(firstRowIds, rowId)) + dataset + .select(firstRowIdUdf(col(identifier))) + .distinct() + .as[Long] + .flatMap( + f => { + if (firstRowIdToBlobFirstRowIds.contains(f)) { + firstRowIdToBlobFirstRowIds(f) + } else { + Seq(f) + } + }) + .collect() + } + + private def extractFields(expression: Expression): Seq[AttributeReference] = { + val fields = new ListBuffer[AttributeReference]() + + def traverse(expr: Expression): Unit = { + expr match { + case attr: AttributeReference => + fields += attr + case other => + other.children.foreach(traverse) + } + } + + traverse(expression) + fields.distinct.toSeq + } + + private def attribute(name: String, plan: LogicalPlan) = + plan.output.find(attr => resolver(name, attr.name)).get + + private def addFirstRowId( + sparkSession: SparkSession, + plan: LogicalPlan, + firstRowIds: immutable.IndexedSeq[Long]): Dataset[Row] = { + assert(plan.output.exists(_.name.equals(ROW_ID_NAME))) + val firstRowIdUdf = udf((rowId: Long) => floorBinarySearch(firstRowIds, rowId)) + val firstRowIdColumn = firstRowIdUdf(col(ROW_ID_NAME)) + createDataset(sparkSession, plan).withColumn(FIRST_ROW_ID_NAME, firstRowIdColumn) + } +} + +object MergeIntoPaimonDataEvolutionTable { + + final private val ROW_FROM_SOURCE = "__row_from_source" + final private val ROW_FROM_TARGET = "__row_from_target" + final private val ROW_ID_NAME = "_ROW_ID" + final private val FIRST_ROW_ID_NAME = "_FIRST_ROW_ID"; + + private def floorBinarySearch(indexed: immutable.IndexedSeq[Long], value: Long): Long = { + if (indexed.isEmpty) { + throw new IllegalArgumentException("The input sorted sequence is empty.") + } + + indexed.search(value) match { + case Found(foundIndex) => indexed(foundIndex) + case InsertionPoint(insertionIndex) => + if (insertionIndex == 0) { + throw new IllegalArgumentException( + s"Value $value is less than the first element in the sorted sequence.") + } else { + indexed(insertionIndex - 1) + } + } + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala new file mode 100644 index 000000000000..9e1f054ec975 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/commands/MergeIntoPaimonTable.scala @@ -0,0 +1,440 @@ +/* + * 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.paimon.spark.commands + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.catalyst.analysis.PaimonRelation +import org.apache.paimon.spark.schema.{PaimonMetadataColumn, SparkSystemColumns} +import org.apache.paimon.spark.schema.PaimonMetadataColumn._ +import org.apache.paimon.spark.util.{EncoderUtils, SparkRowUtils} +import org.apache.paimon.table.{FileStoreTable, SpecialFields} +import org.apache.paimon.table.sink.CommitMessage +import org.apache.paimon.types.RowKind + +import org.apache.spark.sql.{Dataset, Row, SparkSession} +import org.apache.spark.sql.PaimonUtils._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, BasePredicate, Expression, Literal, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.functions.{col, lit, monotonically_increasing_id, sum} +import org.apache.spark.sql.types.{ByteType, StructField, StructType} + +import scala.collection.mutable + +/** Command for Merge Into. */ +case class MergeIntoPaimonTable( + v2Table: SparkTable, + targetTable: LogicalPlan, + sourceTable: LogicalPlan, + mergeCondition: Expression, + matchedActions: Seq[MergeAction], + notMatchedActions: Seq[MergeAction], + notMatchedBySourceActions: Seq[MergeAction]) + extends PaimonRowLevelCommand { + + import MergeIntoPaimonTable._ + + override val table: FileStoreTable = v2Table.getTable.asInstanceOf[FileStoreTable] + + lazy val relation: DataSourceV2Relation = PaimonRelation.getPaimonRelation(targetTable) + + private lazy val (targetOnlyCondition, filteredTargetPlan): (Option[Expression], LogicalPlan) = { + val filtersOnlyTarget = getExpressionOnlyRelated(mergeCondition, targetTable) + ( + filtersOnlyTarget, + filtersOnlyTarget + .map(Filter.apply(_, targetTable)) + .getOrElse(targetTable)) + } + + override def run(sparkSession: SparkSession): Seq[Row] = { + // Avoid that more than one source rows match the same target row. + checkMatchRationality(sparkSession) + val commitMessages = if (withPrimaryKeys) { + performMergeForPkTable(sparkSession) + } else { + performMergeForNonPkTable(sparkSession) + } + writer.commit(commitMessages) + Seq.empty[Row] + } + + private def performMergeForPkTable(sparkSession: SparkSession): Seq[CommitMessage] = { + writer.write( + constructChangedRows( + sparkSession, + createDataset(sparkSession, filteredTargetPlan), + remainDeletedRow = true)) + } + + private def performMergeForNonPkTable(sparkSession: SparkSession): Seq[CommitMessage] = { + // todo: find a more universal way to make read snapshot consistent. + val readSnapshot = table.snapshotManager().latestSnapshot() + val targetDS = createDataset(sparkSession, filteredTargetPlan) + val sourceDS = createDataset(sparkSession, sourceTable) + + // Step1: get the candidate data splits which are filtered by Paimon Predicate. + val candidateDataSplits = + findCandidateDataSplits(targetOnlyCondition.getOrElse(TrueLiteral), relation.output) + val dataFilePathToMeta = candidateFileMap(candidateDataSplits) + + if (deletionVectorsEnabled) { + // Step2: generate dataset that should contains ROW_KIND, FILE_PATH, ROW_INDEX columns + val filteredDf = createDataset( + sparkSession, + createNewScanPlan(candidateDataSplits, relation, targetOnlyCondition)) + val ds = constructChangedRows( + sparkSession, + selectWithDvMeta(filteredDf), + remainDeletedRow = true, + extraMetadataCols = dvMetaCols) + + ds.cache() + try { + // Step3: filter rows that should be marked as DELETED in Deletion Vector mode. + val dvDS = ds.where( + s"$ROW_KIND_COL = ${RowKind.DELETE.toByteValue} or $ROW_KIND_COL = ${RowKind.UPDATE_AFTER.toByteValue}") + val deletionVectors = collectDeletionVectors(dataFilePathToMeta, dvDS, sparkSession) + val indexCommitMsg = writer.persistDeletionVectors(deletionVectors, readSnapshot) + + // Step4: filter rows that should be written as the inserted/updated data. + val toWriteDS = ds + .where( + s"$ROW_KIND_COL = ${RowKind.INSERT.toByteValue} or $ROW_KIND_COL = ${RowKind.UPDATE_AFTER.toByteValue}") + .drop(FILE_PATH_COLUMN, ROW_INDEX_COLUMN) + val addCommitMessage = writer.write(toWriteDS) + + // Step5: commit index and data commit messages + addCommitMessage ++ indexCommitMsg + } finally { + ds.unpersist() + } + } else { + // Files need to be rewritten + val filePathsToRewritten = mutable.Set.empty[String] + // Files need to be read, but not rewritten + val filePathsToRead = mutable.Set.empty[String] + + def hasUpdate(actions: Seq[MergeAction]): Boolean = { + actions.exists { + case _: UpdateAction | _: DeleteAction => true + case _ => false + } + } + + // If there is filter, we need to output the __paimon__file_path metadata column explicitly. + val targetDSWithFilePathCol = targetOnlyCondition.fold(targetDS) { + condition => + createDataset(sparkSession, Filter.apply(condition, relation.withMetadataColumns())) + } + + def findTouchedFiles0(joinType: String): Array[String] = { + findTouchedFiles( + targetDSWithFilePathCol.alias("_left").join(sourceDS, toColumn(mergeCondition), joinType), + sparkSession, + "_left." + FILE_PATH_COLUMN) + } + + if (hasUpdate(matchedActions)) { + filePathsToRewritten ++= findTouchedFiles0("inner") + } else if (notMatchedActions.nonEmpty) { + filePathsToRead ++= findTouchedFiles0("inner") + } + + if (hasUpdate(notMatchedBySourceActions)) { + val noMatchedBySourceFilePaths = findTouchedFiles0("left_anti") + filePathsToRewritten ++= noMatchedBySourceFilePaths + filePathsToRead --= noMatchedBySourceFilePaths + } + + val (filesToRewritten, filesToRewrittenScan) = + extractFilesAndCreateNewScan(filePathsToRewritten.toArray, dataFilePathToMeta, relation) + val (_, filesToReadScan) = + extractFilesAndCreateNewScan(filePathsToRead.toArray, dataFilePathToMeta, relation) + + // If no files need to be rewritten, no need to write row tracking + val writeRowTracking = coreOptions.rowTrackingEnabled() && filesToRewritten.nonEmpty + + // Add FILE_TOUCHED_COL to mark the row as coming from the touched file, if the row has not been + // modified and was from touched file, it should be kept too. + var filesToRewrittenDS = + createDataset(sparkSession, filesToRewrittenScan).withColumn(FILE_TOUCHED_COL, lit(true)) + if (writeRowTracking) { + filesToRewrittenDS = selectWithRowTracking(filesToRewrittenDS) + } + + var filesToReadDS = + createDataset(sparkSession, filesToReadScan).withColumn(FILE_TOUCHED_COL, lit(false)) + if (writeRowTracking) { + // For filesToReadScan we don't need to read row tracking meta cols, just add placeholders + ROW_TRACKING_META_COLUMNS.foreach( + c => filesToReadDS = filesToReadDS.withColumn(c, lit(null))) + } + + val toWriteDS = constructChangedRows( + sparkSession, + filesToRewrittenDS.union(filesToReadDS), + writeRowTracking = writeRowTracking).drop(ROW_KIND_COL) + + val rowTrackingNotNull = col(ROW_ID_COLUMN).isNotNull + val rowTrackingNull = col(ROW_ID_COLUMN).isNull + val addCommitMessageBuilder = Seq.newBuilder[CommitMessage] + if (writeRowTracking) { + val rowTrackingWriter = writer.withRowTracking() + addCommitMessageBuilder ++= rowTrackingWriter.write(toWriteDS.filter(rowTrackingNotNull)) + addCommitMessageBuilder ++= writer.write( + toWriteDS.filter(rowTrackingNull).drop(ROW_ID_COLUMN, SEQUENCE_NUMBER_COLUMN)) + } else { + addCommitMessageBuilder ++= writer.write(toWriteDS) + } + + val addCommitMessage = addCommitMessageBuilder.result() + val deletedCommitMessage = buildDeletedCommitMessage(filesToRewritten) + addCommitMessage ++ deletedCommitMessage + } + } + + /** Get a Dataset where each of Row has an additional column called _row_kind_. */ + private def constructChangedRows( + sparkSession: SparkSession, + targetDataset: Dataset[Row], + remainDeletedRow: Boolean = false, + deletionVectorEnabled: Boolean = false, + extraMetadataCols: Seq[PaimonMetadataColumn] = Seq.empty, + writeRowTracking: Boolean = false): Dataset[Row] = { + val targetDS = targetDataset + .withColumn(TARGET_ROW_COL, lit(true)) + + val sourceDS = createDataset(sparkSession, sourceTable) + .withColumn(SOURCE_ROW_COL, lit(true)) + + val joinedDS = sourceDS.join(targetDS, toColumn(mergeCondition), "fullOuter") + val joinedPlan = joinedDS.queryExecution.analyzed + + def resolveOnJoinedPlan(exprs: Seq[Expression]): Seq[Expression] = { + resolveExpressions(sparkSession)(exprs, joinedPlan) + } + + val targetRowNotMatched = resolveOnJoinedPlan( + Seq(toExpression(sparkSession, col(SOURCE_ROW_COL).isNull))).head + val sourceRowNotMatched = resolveOnJoinedPlan( + Seq(toExpression(sparkSession, col(TARGET_ROW_COL).isNull))).head + val matchedExprs = matchedActions.map(_.condition.getOrElse(TrueLiteral)) + val notMatchedExprs = notMatchedActions.map(_.condition.getOrElse(TrueLiteral)) + val notMatchedBySourceExprs = notMatchedBySourceActions.map(_.condition.getOrElse(TrueLiteral)) + + val resolver = sparkSession.sessionState.conf.resolver + def attribute(name: String) = joinedPlan.output.find(attr => resolver(name, attr.name)) + val extraMetadataAttributes = + extraMetadataCols.flatMap(metadataCol => attribute(metadataCol.name)) + val (rowIdAttr, sequenceNumberAttr) = if (writeRowTracking) { + ( + attribute(SpecialFields.ROW_ID.name()).get, + attribute(SpecialFields.SEQUENCE_NUMBER.name()).get) + } else { + (null, null) + } + + val targetOutput = if (writeRowTracking) { + filteredTargetPlan.output ++ Seq(rowIdAttr, sequenceNumberAttr) + } else { + filteredTargetPlan.output + } + val noopOutput = targetOutput :+ Alias(Literal(NOOP_ROW_KIND_VALUE), ROW_KIND_COL)() + val keepOutput = targetOutput :+ Alias(Literal(RowKind.INSERT.toByteValue), ROW_KIND_COL)() + + def processMergeActions(actions: Seq[MergeAction]): Seq[Seq[Expression]] = { + val columnExprs = actions.map { + case UpdateAction(_, assignments, _) => + var exprs = assignments.map(_.value) + if (writeRowTracking) { + exprs ++= Seq(rowIdAttr, Literal(null)) + } + exprs :+ Literal(RowKind.UPDATE_AFTER.toByteValue) + case DeleteAction(_) => + if (remainDeletedRow || deletionVectorEnabled) { + targetOutput :+ Literal(RowKind.DELETE.toByteValue) + } else { + // If RowKind = NOOP_ROW_KIND_VALUE, then these rows will be dropped in MergeIntoProcessor.processPartition by default. + // If these rows still need to be remained, set MergeIntoProcessor.remainNoopRow true. + noopOutput + } + case InsertAction(_, assignments) => + var exprs = assignments.map(_.value) + if (writeRowTracking) { + exprs ++= Seq(rowIdAttr, sequenceNumberAttr) + } + exprs :+ Literal(RowKind.INSERT.toByteValue) + } + + columnExprs.map(exprs => exprs ++ extraMetadataAttributes) + } + + val matchedOutputs = processMergeActions(matchedActions) + val notMatchedBySourceOutputs = processMergeActions(notMatchedBySourceActions) + val notMatchedOutputs = processMergeActions(notMatchedActions) + val outputFields = mutable.ArrayBuffer(targetTable.schema.fields: _*) + if (writeRowTracking) { + outputFields += PaimonMetadataColumn.ROW_ID.toStructField + outputFields += PaimonMetadataColumn.SEQUENCE_NUMBER.toStructField + } + outputFields += StructField(ROW_KIND_COL, ByteType) + outputFields ++= extraMetadataCols.map(_.toStructField) + val outputSchema = StructType(outputFields.toSeq) + + val joinedRowEncoder = EncoderUtils.encode(joinedPlan.schema) + val outputEncoder = EncoderUtils.encode(outputSchema).resolveAndBind() + + val processor = MergeIntoProcessor( + joinedPlan.output, + targetRowNotMatched, + sourceRowNotMatched, + matchedExprs, + matchedOutputs, + notMatchedBySourceExprs, + notMatchedBySourceOutputs, + notMatchedExprs, + notMatchedOutputs, + noopOutput, + keepOutput, + joinedRowEncoder, + outputEncoder + ) + joinedDS.mapPartitions(processor.processPartition)(outputEncoder).toDF() + } + + private def checkMatchRationality(sparkSession: SparkSession): Unit = { + if (matchedActions.nonEmpty) { + val targetDS = createDataset(sparkSession, filteredTargetPlan) + .withColumn(ROW_ID_COL, monotonically_increasing_id()) + val sourceDS = createDataset(sparkSession, sourceTable) + val count = sourceDS + .join(targetDS, toColumn(mergeCondition), "inner") + .select(col(ROW_ID_COL), lit(1).as("one")) + .groupBy(ROW_ID_COL) + .agg(sum("one").as("count")) + .filter("count > 1") + .count() + if (count > 0) { + throw new RuntimeException( + "Can't execute this MergeInto when there are some target rows that each of " + + "them match more than one source rows. It may lead to an unexpected result.") + } + } + } +} + +object MergeIntoPaimonTable { + private val ROW_ID_COL = "_row_id_" + private val SOURCE_ROW_COL = "_source_row_" + private val TARGET_ROW_COL = "_target_row_" + private val FILE_TOUCHED_COL = "_file_touched_col_" + // +I, +U, -U, -D + private val ROW_KIND_COL: String = SparkSystemColumns.ROW_KIND_COL + private val NOOP_ROW_KIND_VALUE: Byte = "-1".toByte + + private case class MergeIntoProcessor( + joinedAttributes: Seq[Attribute], + targetRowHasNoMatch: Expression, + sourceRowHasNoMatch: Expression, + matchedConditions: Seq[Expression], + matchedOutputs: Seq[Seq[Expression]], + notMatchedBySourceConditions: Seq[Expression], + notMatchedBySourceOutputs: Seq[Seq[Expression]], + notMatchedConditions: Seq[Expression], + notMatchedOutputs: Seq[Seq[Expression]], + noopCopyOutput: Seq[Expression], + keepOutput: Seq[Expression], + joinedRowEncoder: ExpressionEncoder[Row], + outputRowEncoder: ExpressionEncoder[Row] + ) extends Serializable { + + private val rowKindColumnIndex: Int = outputRowEncoder.schema.fieldIndex(ROW_KIND_COL) + + private val fileTouchedColumnIndex: Int = + SparkRowUtils.getFieldIndex(joinedRowEncoder.schema, FILE_TOUCHED_COL) + + private def generateProjection(exprs: Seq[Expression]): UnsafeProjection = { + UnsafeProjection.create(exprs, joinedAttributes) + } + + private def generatePredicate(expr: Expression): BasePredicate = { + GeneratePredicate.generate(expr, joinedAttributes) + } + + private def fromTouchedFile(row: InternalRow): Boolean = { + fileTouchedColumnIndex != -1 && row.getBoolean(fileTouchedColumnIndex) + } + + private def unusedRow(row: InternalRow): Boolean = { + row.getByte(rowKindColumnIndex) == NOOP_ROW_KIND_VALUE + } + + def processPartition(rowIterator: Iterator[Row]): Iterator[Row] = { + val targetRowHasNoMatchPred = generatePredicate(targetRowHasNoMatch) + val sourceRowHasNoMatchPred = generatePredicate(sourceRowHasNoMatch) + val matchedPreds = matchedConditions.map(generatePredicate) + val matchedProjs = matchedOutputs.map(generateProjection) + val notMatchedBySourcePreds = notMatchedBySourceConditions.map(generatePredicate) + val notMatchedBySourceProjs = notMatchedBySourceOutputs.map(generateProjection) + val notMatchedPreds = notMatchedConditions.map(generatePredicate) + val notMatchedProjs = notMatchedOutputs.map(generateProjection) + val noopCopyProj = generateProjection(noopCopyOutput) + val keepProj = generateProjection(keepOutput) + val outputProj = UnsafeProjection.create(outputRowEncoder.schema) + + def processRow(inputRow: InternalRow): InternalRow = { + def applyPreds(preds: Seq[BasePredicate], projs: Seq[UnsafeProjection]): InternalRow = { + preds.zip(projs).find { case (predicate, _) => predicate.eval(inputRow) } match { + case Some((_, projections)) => + projections.apply(inputRow) + case None => + // keep the row if it is from touched file and not be matched + if (fromTouchedFile(inputRow)) { + keepProj.apply(inputRow) + } else { + noopCopyProj.apply(inputRow) + } + } + } + + if (targetRowHasNoMatchPred.eval(inputRow)) { + applyPreds(notMatchedBySourcePreds, notMatchedBySourceProjs) + } else if (sourceRowHasNoMatchPred.eval(inputRow)) { + applyPreds(notMatchedPreds, notMatchedProjs) + } else { + applyPreds(matchedPreds, matchedProjs) + } + } + + val toRow = joinedRowEncoder.createSerializer() + val fromRow = outputRowEncoder.createDeserializer() + rowIterator + .map(toRow) + .map(processRow) + .filterNot(unusedRow) + .map(notDeletedInternalRow => fromRow(outputProj(notDeletedInternalRow))) + } + } + +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala new file mode 100644 index 000000000000..832291e37952 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/paimon/spark/util/ScanPlanHelper.scala @@ -0,0 +1,79 @@ +/* + * 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.paimon.spark.util + +import org.apache.paimon.spark.SparkTable +import org.apache.paimon.spark.schema.PaimonMetadataColumn.{PATH_AND_INDEX_META_COLUMNS, ROW_TRACKING_META_COLUMNS} +import org.apache.paimon.table.{InnerTable, KnownSplitsTable} +import org.apache.paimon.table.source.Split + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.SQLConfHelper +import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions.Literal.TrueLiteral +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.functions.col + +trait ScanPlanHelper extends SQLConfHelper { + + /** Create a new scan plan from a relation with the given data splits, condition(optional). */ + def createNewScanPlan( + dataSplits: Seq[Split], + relation: DataSourceV2Relation, + condition: Option[Expression]): LogicalPlan = { + val newRelation = createNewScanPlan(dataSplits, relation) + condition match { + case Some(c) if c != TrueLiteral => Filter(c, newRelation) + case _ => newRelation + } + } + + def createNewScanPlan( + dataSplits: Seq[Split], + relation: DataSourceV2Relation): DataSourceV2Relation = { + relation.table match { + case sparkTable @ SparkTable(table: InnerTable) => + val knownSplitsTable = KnownSplitsTable.create(table, dataSplits.toArray) + relation.copy(table = sparkTable.copy(table = knownSplitsTable)) + case _ => throw new RuntimeException() + } + } + + def selectWithDvMeta(data: DataFrame): DataFrame = { + selectWithAdditionalCols(data, PATH_AND_INDEX_META_COLUMNS) + } + + def selectWithRowTracking(data: DataFrame): DataFrame = { + selectWithAdditionalCols(data, ROW_TRACKING_META_COLUMNS) + } + + private def selectWithAdditionalCols(data: DataFrame, additionalCols: Seq[String]): DataFrame = { + val dataColNames = data.schema.names + val mergedColNames = dataColNames ++ additionalCols.filterNot(dataColNames.contains) + data.select(mergedColNames.map(col): _*) + } +} + +/** This wrapper is only used in java code, e.g. Procedure. */ +object ScanPlanHelper extends ScanPlanHelper { + def createNewScanPlan(dataSplits: Array[Split], relation: DataSourceV2Relation): LogicalPlan = { + ScanPlanHelper.createNewScanPlan(dataSplits.toSeq, relation) + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala new file mode 100644 index 000000000000..bd5ab95682da --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/catalyst/parser/extensions/RewritePaimonFunctionCommands.scala @@ -0,0 +1,250 @@ +/* + * 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.parser.extensions + +import org.apache.paimon.catalog.Catalog.SYSTEM_DATABASE_NAME +import org.apache.paimon.function.{Function => PaimonFunction} +import org.apache.paimon.function.FunctionDefinition +import org.apache.paimon.spark.SparkCatalog.FUNCTION_DEFINITION_NAME +import org.apache.paimon.spark.catalog.SupportV1Function +import org.apache.paimon.spark.catalog.functions.PaimonFunctions +import org.apache.paimon.spark.execution.{CreatePaimonV1FunctionCommand, DescribePaimonV1FunctionCommand, DropPaimonV1FunctionCommand} +import org.apache.paimon.spark.util.OptionUtils + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.{UnresolvedException, UnresolvedFunction, UnresolvedFunctionName, UnresolvedIdentifier} +import org.apache.spark.sql.catalyst.catalog.CatalogFunction +import org.apache.spark.sql.catalyst.expressions.{Expression, Unevaluable} +import org.apache.spark.sql.catalyst.plans.logical.{CreateFunction, DescribeFunction, DropFunction, LogicalPlan, SubqueryAlias, UnresolvedWith} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.trees.TreePattern.{TreePattern, UNRESOLVED_FUNCTION} +import org.apache.spark.sql.connector.catalog.{CatalogManager, CatalogPlugin, LookupCatalog} +import org.apache.spark.sql.types.DataType + +/** + * Spark 4.1 shim for RewritePaimonFunctionCommands. + * + * Two incompatibilities with Spark 4.1.1: + * + * 1. UnResolvedPaimonV1Function extends Unevaluable. In Spark 4.0.2, Unevaluable extends + * FoldableUnevaluable. In Spark 4.1.1, FoldableUnevaluable was removed entirely, so the class + * compiled against 4.0.2 fails with ClassNotFoundException at runtime. + * 2. UnresolvedWith.cteRelations changed from Seq[(String, SubqueryAlias)] in 4.0.2 to + * Seq[(String, SubqueryAlias, Option[Boolean])] in 4.1.1. The transformPaimonV1Function method + * must preserve the third tuple element. + */ +case class RewritePaimonFunctionCommands(spark: SparkSession) + extends Rule[LogicalPlan] + with LookupCatalog { + + protected lazy val catalogManager: CatalogManager = spark.sessionState.catalogManager + + override def apply(plan: LogicalPlan): LogicalPlan = { + // Add a global switch to enable/disable v1 function. + if (!OptionUtils.v1FunctionEnabled()) { + return plan + } + + val applied = plan.resolveOperatorsUp { + case CreateFunction(CatalogAndFunctionIdentifier(_, funcIdent, true), _, _, _, replace) => + if (replace) { + throw new UnsupportedOperationException( + s"$funcIdent is a temporary function, you should use `CREATE OR REPLACE TEMPORARY FUNCTION $funcIdent` or DROP TEMPORARY FUNCTION $funcIdent`.") + } else { + throw new UnsupportedOperationException( + s"$funcIdent is a temporary function and already exists.") + } + + case CreateFunction( + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), + className, + resources, + ifExists, + replace) => + if (isPaimonBuildInFunction(funcIdent)) { + throw new UnsupportedOperationException(s"Can't create build-in function: $funcIdent") + } + val v1Function = CatalogFunction(funcIdent, className, resources) + CreatePaimonV1FunctionCommand(v1FunctionCatalog, v1Function, ifExists, replace) + + case DropFunction( + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), + ifExists) => + if (isPaimonBuildInFunction(funcIdent)) { + throw new UnsupportedOperationException(s"Can't drop build-in function: $funcIdent") + } + // The function may be v1 function or not, anyway it can be safely deleted here. + DropPaimonV1FunctionCommand(v1FunctionCatalog, funcIdent, ifExists) + + case d @ DescribeFunction( + CatalogAndFunctionIdentifier(v1FunctionCatalog: SupportV1Function, funcIdent, false), + isExtended) + // For Paimon built-in functions, Spark will resolve them by itself. + if !isPaimonBuildInFunction(funcIdent) => + val function = v1FunctionCatalog.getFunction(funcIdent) + if (isPaimonV1Function(function)) { + DescribePaimonV1FunctionCommand(function, isExtended) + } else { + d + } + } + + // Needs to be done here and transform to `UnResolvedPaimonV1Function`, so that spark's Analyzer can resolve + // the 'arguments' without throwing an exception, saying that function is not supported. + transformPaimonV1Function(applied) + } + + private def transformPaimonV1Function(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsUp { + case u: UnresolvedWith => + // In Spark 4.1.1, cteRelations is Seq[(String, SubqueryAlias, Option[Boolean])]. + // Preserve the third element (allowRecursion flag) when transforming. + u.copy(cteRelations = u.cteRelations.map( + t => (t._1, transformPaimonV1Function(t._2).asInstanceOf[SubqueryAlias], t._3))) + case l: LogicalPlan => + l.transformExpressionsWithPruning(_.containsAnyPattern(UNRESOLVED_FUNCTION)) { + case u: UnresolvedFunction => + CatalogAndFunctionIdentifier.unapply(u.nameParts) match { + case Some((v1FunctionCatalog: SupportV1Function, funcIdent, false)) + // For Paimon built-in functions, Spark will resolve them by itself. + if !isPaimonBuildInFunction(funcIdent) => + // If the function is already registered, avoid redundant lookup in the catalog to reduce overhead. + if (v1FunctionCatalog.v1FunctionRegistered(funcIdent)) { + UnResolvedPaimonV1Function(funcIdent, u, None) + } else { + val function = v1FunctionCatalog.getFunction(funcIdent) + if (isPaimonV1Function(function)) { + UnResolvedPaimonV1Function(funcIdent, u, Some(function)) + } else { + u + } + } + case _ => u + } + } + } + } + + private object CatalogAndFunctionIdentifier { + + def unapply(unresolved: LogicalPlan): Option[(CatalogPlugin, FunctionIdentifier, Boolean)] = + unresolved match { + case ui: UnresolvedIdentifier => + unapply(ui.nameParts) + case name: UnresolvedFunctionName => + unapply(name.multipartIdentifier) + case _ => + None + } + + def unapply(nameParts: Seq[String]): Option[(CatalogPlugin, FunctionIdentifier, Boolean)] = { + nameParts match { + // Spark's built-in or tmp functions is without database name or catalog name. + case Seq(funName) if isSparkBuiltInFunction(FunctionIdentifier(funName)) => + None + case Seq(funName) if isSparkTmpFunc(FunctionIdentifier(funName)) => + Some(null, FunctionIdentifier(funName), true) + case CatalogAndIdentifier(v1FunctionCatalog: SupportV1Function, ident) + if v1FunctionCatalog.v1FunctionEnabled() => + Some( + v1FunctionCatalog, + FunctionIdentifier( + ident.name(), + Some(ident.namespace().last), + Some(v1FunctionCatalog.name)), + false) + case _ => + None + } + } + } + + private def isPaimonBuildInFunction(funcIdent: FunctionIdentifier): Boolean = { + funcIdent.database match { + case Some(db) + if db == SYSTEM_DATABASE_NAME && PaimonFunctions.names.contains(funcIdent.funcName) => + true + case _ => false + } + } + + private def isSparkBuiltInFunction(funcIdent: FunctionIdentifier): Boolean = { + catalogManager.v1SessionCatalog.isBuiltinFunction(funcIdent) + } + + private def isSparkTmpFunc(funcIdent: FunctionIdentifier): Boolean = { + catalogManager.v1SessionCatalog.isTemporaryFunction(funcIdent) + } + + private def isPaimonV1Function(fun: PaimonFunction): Boolean = { + fun.definition(FUNCTION_DEFINITION_NAME) match { + case _: FunctionDefinition.FileFunctionDefinition => true + case _ => false + } + } +} + +/** An unresolved Paimon V1 function to let Spark resolve the necessary variables. */ +case class UnResolvedPaimonV1Function( + funcIdent: FunctionIdentifier, + arguments: Seq[Expression], + isDistinct: Boolean, + filter: Option[Expression] = None, + ignoreNulls: Boolean = false, + func: Option[PaimonFunction] = None) + extends Expression + with Unevaluable { + + override def children: Seq[Expression] = arguments ++ filter.toSeq + + override def dataType: DataType = throw new UnresolvedException("dataType") + + override def nullable: Boolean = throw new UnresolvedException("nullable") + + override lazy val resolved = false + + final override val nodePatterns: Seq[TreePattern] = Seq(UNRESOLVED_FUNCTION) + + override def prettyName: String = funcIdent.identifier + + override def toString: String = { + val distinct = if (isDistinct) "distinct " else "" + s"'$prettyName($distinct${children.mkString(", ")})" + } + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): UnResolvedPaimonV1Function = { + if (filter.isDefined) { + copy(arguments = newChildren.dropRight(1), filter = Some(newChildren.last)) + } else { + copy(arguments = newChildren) + } + } +} + +object UnResolvedPaimonV1Function { + + def apply( + funcIdent: FunctionIdentifier, + u: UnresolvedFunction, + fun: Option[PaimonFunction]): UnResolvedPaimonV1Function = { + UnResolvedPaimonV1Function(funcIdent, u.arguments, u.isDistinct, u.filter, u.ignoreNulls, fun) + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/SparkFormatTable.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/SparkFormatTable.scala new file mode 100644 index 000000000000..8aa5fd71ff4d --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/SparkFormatTable.scala @@ -0,0 +1,332 @@ +/* + * 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 + +import org.apache.paimon.utils.StringUtils + +import org.apache.hadoop.fs.Path +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.connector.catalog.SupportsPartitionManagement +import org.apache.spark.sql.connector.catalog.TableCapability +import org.apache.spark.sql.connector.catalog.TableCapability._ +import org.apache.spark.sql.connector.expressions.{Expressions, Transform} +import org.apache.spark.sql.connector.expressions.filter.Predicate +import org.apache.spark.sql.connector.write.{LogicalWriteInfo, SupportsOverwriteV2, Write, WriteBuilder} +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.v2.csv.{CSVScanBuilder, CSVTable} +import org.apache.spark.sql.execution.datasources.v2.json.JsonTable +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable +import org.apache.spark.sql.execution.datasources.v2.text.{TextScanBuilder, TextTable} +import org.apache.spark.sql.execution.streaming.runtime.MetadataLogFileIndex +import org.apache.spark.sql.execution.streaming.sinks.FileStreamSink +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +import java.util + +import scala.collection.JavaConverters._ + +/** + * Spark 4.1 shim for SparkFormatTable. + * + * In Spark 4.1.1, FileStreamSink moved from org.apache.spark.sql.execution.streaming to + * org.apache.spark.sql.execution.streaming.sinks, and MetadataLogFileIndex moved from + * org.apache.spark.sql.execution.streaming to org.apache.spark.sql.execution.streaming.runtime. + */ +object SparkFormatTable { + + // Copy from spark and override FileIndex's partitionSchema + def createFileIndex( + options: CaseInsensitiveStringMap, + sparkSession: SparkSession, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + partitionSchema: StructType): PartitioningAwareFileIndex = { + + def globPaths: Boolean = { + val entry = options.get(DataSource.GLOB_PATHS_KEY) + Option(entry).forall(_ == "true") + } + + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case-sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + if (FileStreamSink.hasMetadata(paths, hadoopConf, sparkSession.sessionState.conf)) { + // We are reading from the results of a streaming query. We will load files from + // the metadata log instead of listing them using HDFS APIs. + new PartitionedMetadataLogFileIndex( + sparkSession, + new Path(paths.head), + options.asScala.toMap, + userSpecifiedSchema, + partitionSchema = partitionSchema) + } else { + // This is a non-streaming file based datasource. + val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary( + paths, + hadoopConf, + checkEmptyGlobPath = true, + checkFilesExist = true, + enableGlobbing = globPaths) + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + + new PartitionedInMemoryFileIndex( + sparkSession, + rootPathsSpecified, + caseSensitiveMap, + userSpecifiedSchema, + fileStatusCache, + partitionSchema = partitionSchema) + } + } + + // Extend from MetadataLogFileIndex to override partitionSchema + private class PartitionedMetadataLogFileIndex( + sparkSession: SparkSession, + path: Path, + parameters: Map[String, String], + userSpecifiedSchema: Option[StructType], + override val partitionSchema: StructType) + extends MetadataLogFileIndex(sparkSession, path, parameters, userSpecifiedSchema) + + // Extend from InMemoryFileIndex to override partitionSchema + private class PartitionedInMemoryFileIndex( + sparkSession: SparkSession, + rootPathsSpecified: Seq[Path], + parameters: Map[String, String], + userSpecifiedSchema: Option[StructType], + fileStatusCache: FileStatusCache = NoopCache, + userSpecifiedPartitionSpec: Option[PartitionSpec] = None, + metadataOpsTimeNs: Option[Long] = None, + override val partitionSchema: StructType) + extends InMemoryFileIndex( + sparkSession, + rootPathsSpecified, + parameters, + userSpecifiedSchema, + fileStatusCache, + userSpecifiedPartitionSpec, + metadataOpsTimeNs) +} + +trait PartitionedFormatTable extends SupportsPartitionManagement { + + val partitionSchema_ : StructType + + val fileIndex: PartitioningAwareFileIndex + + override def capabilities(): util.Set[TableCapability] = { + util.EnumSet.of(BATCH_READ, BATCH_WRITE, OVERWRITE_DYNAMIC, OVERWRITE_BY_FILTER) + } + + protected def wrapWriteBuilderWithOverwrite(original: WriteBuilder): WriteBuilder = { + new WriteBuilder with SupportsOverwriteV2 { + override def build(): Write = original.build() + override def canOverwrite(predicates: Array[Predicate]): Boolean = true + override def overwrite(predicates: Array[Predicate]): WriteBuilder = this + } + } + + override def partitionSchema(): StructType = partitionSchema_ + + override def partitioning(): Array[Transform] = { + partitionSchema().fields.map(f => Expressions.identity(StringUtils.quote(f.name))).toArray + } + + override def listPartitionIdentifiers( + names: Array[String], + ident: InternalRow): Array[InternalRow] = { + val partitionFilters = names.zipWithIndex.map { + case (name, index) => + val f = partitionSchema().apply(name) + EqualTo( + AttributeReference(f.name, f.dataType, f.nullable)(), + Literal(ident.get(index, f.dataType), f.dataType)) + }.toSeq + fileIndex.listFiles(partitionFilters, Seq.empty).map(_.values).toArray + } + + override def createPartition(ident: InternalRow, properties: util.Map[String, String]): Unit = { + throw new UnsupportedOperationException() + } + + override def dropPartition(ident: InternalRow): Boolean = { + throw new UnsupportedOperationException() + } + + override def replacePartitionMetadata( + ident: InternalRow, + properties: util.Map[String, String]): Unit = { + throw new UnsupportedOperationException() + } + + override def loadPartitionMetadata(ident: InternalRow): util.Map[String, String] = { + Map.empty[String, String].asJava + } +} + +class PartitionedCSVTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat], + override val partitionSchema_ : StructType) + extends CSVTable(name, sparkSession, options, paths, userSpecifiedSchema, fallbackFileFormat) + with PartitionedFormatTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + wrapWriteBuilderWithOverwrite(super.newWriteBuilder(info)) + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): CSVScanBuilder = { + val mergedOptions = + this.options.asCaseSensitiveMap().asScala ++ options.asCaseSensitiveMap().asScala + CSVScanBuilder( + sparkSession, + fileIndex, + schema, + dataSchema, + new CaseInsensitiveStringMap(mergedOptions.asJava)) + } + + override lazy val fileIndex: PartitioningAwareFileIndex = { + SparkFormatTable.createFileIndex( + options, + sparkSession, + paths, + userSpecifiedSchema, + partitionSchema()) + } +} + +class PartitionedTextTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat], + override val partitionSchema_ : StructType) + extends TextTable(name, sparkSession, options, paths, userSpecifiedSchema, fallbackFileFormat) + with PartitionedFormatTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + wrapWriteBuilderWithOverwrite(super.newWriteBuilder(info)) + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): TextScanBuilder = { + val mergedOptions = + this.options.asCaseSensitiveMap().asScala ++ options.asCaseSensitiveMap().asScala + TextScanBuilder( + sparkSession, + fileIndex, + schema, + dataSchema, + new CaseInsensitiveStringMap(mergedOptions.asJava)) + } + + override lazy val fileIndex: PartitioningAwareFileIndex = { + SparkFormatTable.createFileIndex( + options, + sparkSession, + paths, + userSpecifiedSchema, + partitionSchema()) + } +} + +class PartitionedOrcTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat], + override val partitionSchema_ : StructType +) extends OrcTable(name, sparkSession, options, paths, userSpecifiedSchema, fallbackFileFormat) + with PartitionedFormatTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + wrapWriteBuilderWithOverwrite(super.newWriteBuilder(info)) + } + + override lazy val fileIndex: PartitioningAwareFileIndex = { + SparkFormatTable.createFileIndex( + options, + sparkSession, + paths, + userSpecifiedSchema, + partitionSchema()) + } +} + +class PartitionedParquetTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat], + override val partitionSchema_ : StructType +) extends ParquetTable(name, sparkSession, options, paths, userSpecifiedSchema, fallbackFileFormat) + with PartitionedFormatTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + wrapWriteBuilderWithOverwrite(super.newWriteBuilder(info)) + } + + override lazy val fileIndex: PartitioningAwareFileIndex = { + SparkFormatTable.createFileIndex( + options, + sparkSession, + paths, + userSpecifiedSchema, + partitionSchema()) + } +} + +class PartitionedJsonTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat], + override val partitionSchema_ : StructType) + extends JsonTable(name, sparkSession, options, paths, userSpecifiedSchema, fallbackFileFormat) + with PartitionedFormatTable { + + override def newWriteBuilder(info: LogicalWriteInfo): WriteBuilder = { + wrapWriteBuilderWithOverwrite(super.newWriteBuilder(info)) + } + + override lazy val fileIndex: PartitioningAwareFileIndex = { + SparkFormatTable.createFileIndex( + options, + sparkSession, + paths, + userSpecifiedSchema, + partitionSchema()) + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala new file mode 100644 index 000000000000..4836f4b11263 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/execution/shim/PaimonCreateTableAsSelectStrategy.scala @@ -0,0 +1,95 @@ +/* + * 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.shim + +import org.apache.paimon.CoreOptions +import org.apache.paimon.iceberg.IcebergOptions +import org.apache.paimon.spark.SparkCatalog +import org.apache.paimon.spark.catalog.FormatTableCatalog + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.analysis.ResolvedIdentifier +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LogicalPlan, TableSpec} +import org.apache.spark.sql.connector.catalog.StagingTableCatalog +import org.apache.spark.sql.execution.{PaimonStrategyHelper, SparkPlan, SparkStrategy} +import org.apache.spark.sql.execution.datasources.v2.CreateTableAsSelectExec + +import scala.collection.JavaConverters._ + +case class PaimonCreateTableAsSelectStrategy(spark: SparkSession) + extends SparkStrategy + with PaimonStrategyHelper { + + import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._ + + // Override to recompile against Spark 4.1's TableSpec.copy (9 params) + override protected def qualifyLocInTableSpec(tableSpec: TableSpec): TableSpec = { + tableSpec.copy(location = tableSpec.location.map(makeQualifiedDBObjectPath(_))) + } + + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case CreateTableAsSelect( + ResolvedIdentifier(catalog: SparkCatalog, ident), + parts, + query, + tableSpec: TableSpec, + options, + ifNotExists, + true) => + catalog match { + case _: StagingTableCatalog => + throw new RuntimeException("Paimon can't extend StagingTableCatalog for now.") + case _ => + val coreOptionKeys = CoreOptions.getOptions.asScala.map(_.key()).toSeq + + // Include Iceberg compatibility options in table properties (fix for DataFrame writer options) + val icebergOptionKeys = IcebergOptions.getOptions.asScala.map(_.key()).toSeq + + val allTableOptionKeys = coreOptionKeys ++ icebergOptionKeys + + val (tableOptions, writeOptions) = options.partition { + case (key, _) => allTableOptionKeys.contains(key) + } + val newTableSpec = tableSpec.copy(properties = tableSpec.properties ++ tableOptions) + + val isPartitionedFormatTable = { + catalog match { + case catalog: FormatTableCatalog => + catalog.isFormatTable(newTableSpec.provider.orNull) && parts.nonEmpty + case _ => false + } + } + + if (isPartitionedFormatTable) { + throw new UnsupportedOperationException( + "Using CTAS with partitioned format table is not supported yet.") + } + + CreateTableAsSelectExec( + catalog.asTableCatalog, + ident, + parts, + query, + qualifyLocInTableSpec(newTableSpec), + writeOptions, + ifNotExists) :: Nil + } + case _ => Nil + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala new file mode 100644 index 000000000000..f0a3cc465a40 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/main/scala/org/apache/spark/sql/paimon/shims/Spark4Shim.scala @@ -0,0 +1,143 @@ +/* + * 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.paimon.shims + +import org.apache.paimon.data.variant.{GenericVariant, Variant} +import org.apache.paimon.spark.catalyst.analysis.Spark4ResolutionRules +import org.apache.paimon.spark.catalyst.parser.extensions.PaimonSpark4SqlExtensionsParser +import org.apache.paimon.spark.data.{Spark4ArrayData, Spark4InternalRow, Spark4InternalRowWithBlob, SparkArrayData, SparkInternalRow} +import org.apache.paimon.types.{DataType, RowType} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.parser.ParserInterface +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, CTERelationRef, LogicalPlan, MergeAction, MergeIntoTable} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.connector.catalog.{CatalogV2Util, Identifier, Table, TableCatalog} +import org.apache.spark.sql.connector.expressions.Transform +import org.apache.spark.sql.types.{DataTypes, StructType, VariantType} +import org.apache.spark.unsafe.types.VariantVal + +import java.util.{Map => JMap} + +/** + * Spark 4.1 shim for Spark4Shim. + * + * In Spark 4.1, CTERelationRef gained two new parameters (maxDepth, isRecursive), going from 6 to 8 + * params. The base Spark4Shim in paimon-spark4-common was compiled against Spark 4.0.2's + * 6-parameter CTERelationRef, causing NoSuchMethodError at runtime. This shim recompiles against + * Spark 4.1.1. + */ +class Spark4Shim extends SparkShim { + + override def classicApi: ClassicApi = new Classic4Api + + override def createSparkParser(delegate: ParserInterface): ParserInterface = { + new PaimonSpark4SqlExtensionsParser(delegate) + } + + override def createCustomResolution(spark: SparkSession): Rule[LogicalPlan] = { + Spark4ResolutionRules(spark) + } + + override def createSparkInternalRow(rowType: RowType): SparkInternalRow = { + new Spark4InternalRow(rowType) + } + + override def createSparkInternalRowWithBlob( + rowType: RowType, + blobFields: Set[Int], + blobAsDescriptor: Boolean): SparkInternalRow = { + new Spark4InternalRowWithBlob(rowType, blobFields, blobAsDescriptor) + } + + override def createSparkArrayData(elementType: DataType): SparkArrayData = { + new Spark4ArrayData(elementType) + } + + override def createTable( + tableCatalog: TableCatalog, + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: JMap[String, String]): Table = { + val columns = CatalogV2Util.structTypeToV2Columns(schema) + tableCatalog.createTable(ident, columns, partitions, properties) + } + + override def createCTERelationRef( + cteId: Long, + resolved: Boolean, + output: Seq[Attribute], + isStreaming: Boolean): CTERelationRef = { + CTERelationRef(cteId, resolved, output.toSeq, isStreaming) + } + + override def supportsHashAggregate( + aggregateBufferAttributes: Seq[Attribute], + groupingExpression: Seq[Expression]): Boolean = { + Aggregate.supportsHashAggregate(aggregateBufferAttributes.toSeq, groupingExpression.toSeq) + } + + override def supportsObjectHashAggregate( + aggregateExpressions: Seq[AggregateExpression], + groupByExpressions: Seq[Expression]): Boolean = + Aggregate.supportsObjectHashAggregate(aggregateExpressions.toSeq, groupByExpressions.toSeq) + + override def createMergeIntoTable( + targetTable: LogicalPlan, + sourceTable: LogicalPlan, + mergeCondition: Expression, + matchedActions: Seq[MergeAction], + notMatchedActions: Seq[MergeAction], + notMatchedBySourceActions: Seq[MergeAction], + withSchemaEvolution: Boolean): MergeIntoTable = { + MergeIntoTable( + targetTable, + sourceTable, + mergeCondition, + matchedActions, + notMatchedActions, + notMatchedBySourceActions, + withSchemaEvolution) + } + + override def toPaimonVariant(o: Object): Variant = { + val v = o.asInstanceOf[VariantVal] + new GenericVariant(v.getValue, v.getMetadata) + } + + override def toPaimonVariant(row: InternalRow, pos: Int): Variant = { + val v = row.getVariant(pos) + new GenericVariant(v.getValue, v.getMetadata) + } + + override def toPaimonVariant(array: ArrayData, pos: Int): Variant = { + val v = array.getVariant(pos) + new GenericVariant(v.getValue, v.getMetadata) + } + + override def isSparkVariantType(dataType: org.apache.spark.sql.types.DataType): Boolean = + dataType.isInstanceOf[VariantType] + + override def SparkVariantType(): org.apache.spark.sql.types.DataType = DataTypes.VariantType +} diff --git a/paimon-spark/paimon-spark-4.1/src/test/resources/function/hive-test-udfs.jar b/paimon-spark/paimon-spark-4.1/src/test/resources/function/hive-test-udfs.jar new file mode 100644 index 000000000000..a5bfa456f668 Binary files /dev/null and b/paimon-spark/paimon-spark-4.1/src/test/resources/function/hive-test-udfs.jar differ diff --git a/paimon-spark/paimon-spark-4.1/src/test/resources/hive-site.xml b/paimon-spark/paimon-spark-4.1/src/test/resources/hive-site.xml new file mode 100644 index 000000000000..bdf2bb090760 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/resources/hive-site.xml @@ -0,0 +1,56 @@ + + + + + hive.metastore.integral.jdo.pushdown + true + + + + hive.metastore.schema.verification + false + + + + hive.metastore.client.capability.check + false + + + + datanucleus.schema.autoCreateTables + true + + + + datanucleus.schema.autoCreateAll + true + + + + + datanucleus.connectionPoolingType + DBCP + + + + hive.metastore.uris + thrift://localhost:9090 + Thrift URI for the remote metastore. Used by metastore client to connect to remote metastore. + + \ No newline at end of file diff --git a/paimon-spark/paimon-spark-4.1/src/test/resources/log4j2-test.properties b/paimon-spark/paimon-spark-4.1/src/test/resources/log4j2-test.properties new file mode 100644 index 000000000000..6f324f5863ac --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/resources/log4j2-test.properties @@ -0,0 +1,38 @@ +################################################################################ +# 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. +################################################################################ + +# Set root logger level to OFF to not flood build logs +# set manually to INFO for debugging purposes +rootLogger.level = OFF +rootLogger.appenderRef.test.ref = TestLogger + +appender.testlogger.name = TestLogger +appender.testlogger.type = CONSOLE +appender.testlogger.target = SYSTEM_ERR +appender.testlogger.layout.type = PatternLayout +appender.testlogger.layout.pattern = %-4r [%tid %t] %-5p %c %x - %m%n + +logger.kafka.name = kafka +logger.kafka.level = OFF +logger.kafka2.name = state.change +logger.kafka2.level = OFF + +logger.zookeeper.name = org.apache.zookeeper +logger.zookeeper.level = OFF +logger.I0Itec.name = org.I0Itec +logger.I0Itec.level = OFF diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.scala new file mode 100644 index 000000000000..29f49270b8d3 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/CompactProcedureTest.scala @@ -0,0 +1,56 @@ +/* + * 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.paimon.spark.procedure + +import org.scalactic.source.Position +import org.scalatest.Tag + +/** + * Spark 4.1 version of CompactProcedureTest. + * + * In Spark 4.1.1, MemoryStream was relocated from + * org.apache.spark.sql.execution.streaming.MemoryStream to + * org.apache.spark.sql.execution.streaming.runtime.MemoryStream. CompactProcedureTestBase in + * paimon-spark-ut was compiled against Spark 4.0.2 and its bytecode references the old package + * path, causing NoClassDefFoundError at runtime. Tests that use MemoryStream are excluded here. + */ +class CompactProcedureTest extends CompactProcedureTestBase { + + // Tests that use MemoryStream (relocated in Spark 4.1.1) are excluded to prevent + // NoClassDefFoundError from aborting the entire test suite. + // Must be a def (not val) because test() is called during parent constructor init, + // before subclass fields are initialized. + private def streamingTests: Set[String] = Set( + "Paimon Procedure: sort compact", + "Paimon Procedure: sort compact with partition", + "Paimon Procedure: compact for pk", + "Paimon Procedure: cluster for unpartitioned table", + "Paimon Procedure: cluster for partitioned table", + "Paimon Procedure: cluster with deletion vectors" + ) + + override def test(testName: String, testTags: Tag*)(testFun: => Any)(implicit + pos: Position): Unit = { + if (streamingTests.contains(testName)) { + super.ignore(testName, testTags: _*)(testFun) + } else { + super.test(testName, testTags: _*)(testFun) + } + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.scala new file mode 100644 index 000000000000..d57846709877 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/procedure/ProcedureTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.procedure + +class ProcedureTest extends ProcedureTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.scala new file mode 100644 index 000000000000..255906d04bf2 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/AnalyzeTableTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class AnalyzeTableTest extends AnalyzeTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLTest.scala new file mode 100644 index 000000000000..b729f57b33e7 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class DDLTest extends DDLTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.scala new file mode 100644 index 000000000000..cb139d2a57be --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DDLWithHiveCatalogTest.scala @@ -0,0 +1,23 @@ +/* + * 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.paimon.spark.sql + +class DDLWithHiveCatalogTest extends DDLWithHiveCatalogTestBase {} + +class DefaultDatabaseTest extends DefaultDatabaseTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala new file mode 100644 index 000000000000..6170e2fd6c5c --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DataFrameWriteTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class DataFrameWriteTest extends DataFrameWriteTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.scala new file mode 100644 index 000000000000..a6b87268b0ea --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DeleteFromTableTest.scala @@ -0,0 +1,31 @@ +/* + * 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.paimon.spark.sql + +import org.apache.spark.SparkConf + +class DeleteFromTableTest extends DeleteFromTableTestBase { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} + +// V2 row-level operations (delete/update/merge) are not supported in Spark 4.1 because +// RewriteDeleteFromTable moved into the Resolution batch and runs before Paimon's post-hoc rules. +// SparkTable does not implement SupportsRowLevelOperations in the 4.1 shim to avoid this conflict. diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala new file mode 100644 index 000000000000..c6aa77419241 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/DescribeTableTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class DescribeTableTest extends DescribeTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/FormatTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/FormatTableTest.scala new file mode 100644 index 000000000000..ba49976ab6c0 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/FormatTableTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class FormatTableTest extends FormatTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala new file mode 100644 index 000000000000..4f66584c303b --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/InsertOverwriteTableTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class InsertOverwriteTableTest extends InsertOverwriteTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala new file mode 100644 index 000000000000..c83ee5493867 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/MergeIntoTableTest.scala @@ -0,0 +1,63 @@ +/* + * 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.paimon.spark.sql + +import org.apache.paimon.spark.{PaimonAppendBucketedTableTest, PaimonAppendNonBucketTableTest, PaimonPrimaryKeyBucketedTableTest, PaimonPrimaryKeyNonBucketTableTest} + +import org.apache.spark.SparkConf + +class MergeIntoPrimaryKeyBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoPrimaryKeyTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonPrimaryKeyBucketedTableTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} + +class MergeIntoPrimaryKeyNonBucketTableTest + extends MergeIntoTableTestBase + with MergeIntoPrimaryKeyTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonPrimaryKeyNonBucketTableTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} + +class MergeIntoAppendBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoAppendTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonAppendBucketedTableTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} + +class MergeIntoAppendNonBucketedTableTest + extends MergeIntoTableTestBase + with MergeIntoAppendTableTest + with MergeIntoNotMatchedBySourceTest + with PaimonAppendNonBucketTableTest { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.scala new file mode 100644 index 000000000000..635185a9ed0e --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonCompositePartitionKeyTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class PaimonCompositePartitionKeyTest extends PaimonCompositePartitionKeyTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala new file mode 100644 index 000000000000..54812eeb4d26 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonOptimizationTest.scala @@ -0,0 +1,256 @@ +/* + * 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.paimon.spark.sql + +import org.apache.paimon.Snapshot.CommitKind +import org.apache.paimon.spark.PaimonSparkTestBase +import org.apache.paimon.spark.catalyst.analysis.expressions.ExpressionHelper +import org.apache.paimon.spark.catalyst.optimizer.MergePaimonScalarSubqueries +import org.apache.paimon.spark.execution.TruncatePaimonTableWithFilterExec + +import org.apache.spark.sql.{DataFrame, PaimonUtils, Row} +import org.apache.spark.sql.{functions => fn} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, CreateNamedStruct, GetStructField, Literal, NamedExpression, ScalarSubquery} +import org.apache.spark.sql.catalyst.plans.logical.{CTERelationDef, LogicalPlan, OneRowRelation, WithCTE} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.CommandResultExec +import org.apache.spark.sql.paimon.shims.SparkShimLoader +import org.junit.jupiter.api.Assertions + +import scala.collection.immutable + +/** + * Spark 4.1 version of PaimonOptimizationTest. + * + * In Spark 4.1, CTERelationDef gained a 5th parameter (maxDepth). The base class + * PaimonOptimizationTestBase in paimon-spark-ut was compiled against Spark 4.0.2's 4-parameter + * CTERelationDef. Since the `definitionNode` method is private and cannot be overridden, this test + * class reimplements the tests directly with the correct CTERelationDef constructor. + */ +class PaimonOptimizationTest extends PaimonSparkTestBase with ExpressionHelper { + + import org.apache.spark.sql.catalyst.dsl.plans._ + import testImplicits._ + + private object Optimize extends RuleExecutor[LogicalPlan] { + val batches: immutable.Seq[Batch] = + Batch("MergePaimonScalarSubqueries", Once, MergePaimonScalarSubqueries) :: Nil + } + + private def definitionNode(plan: LogicalPlan, cteIndex: Int) = { + CTERelationDef(plan, cteIndex, underSubquery = true) + } + + private def extractorExpression( + cteIndex: Int, + output: Seq[Attribute], + fieldIndex: Int): NamedExpression = { + GetStructField( + ScalarSubquery( + SparkShimLoader.shim + .createCTERelationRef(cteIndex, resolved = true, output.toSeq, isStreaming = false)), + fieldIndex, + None) + .as("scalarsubquery()") + } + + test("Paimon Optimization: merge scalar subqueries") { + withTable("T") { + + spark.sql(s""" + |CREATE TABLE T (a INT, b DOUBLE, c STRING) + |""".stripMargin) + + spark.sql("INSERT INTO T values (1, 11.1, 'x1'), (2, 22.2, 'x2'), (3, 33.3, 'x3')") + + val query = spark.sql(s""" + |SELECT + | (SELECT COUNT(1) AS cnt FROM T), + | (SELECT SUM(a) AS sum_a FROM T), + | (SELECT AVG(b) AS avg_b FROM T) + |""".stripMargin) + val optimizedPlan = Optimize.execute(query.queryExecution.analyzed) + val id = optimizedPlan.asInstanceOf[WithCTE].cteDefs.head.id.toInt + + val df = PaimonUtils.createDataset(spark, createRelationV2("T")) + val mergedSubquery = df + .select( + toColumn(count(Literal(1))).as("cnt"), + toColumn(sum(toExpression(spark, fn.col("a")))).as("sum_a"), + toColumn(avg(toExpression(spark, fn.col("b"))).as("avg_b")) + ) + .select( + toColumn( + CreateNamedStruct( + Seq( + Literal("cnt"), + 'cnt, + Literal("sum_a"), + 'sum_a, + Literal("avg_b"), + 'avg_b + )).as("mergedValue"))) + val analyzedMergedSubquery = mergedSubquery.queryExecution.analyzed + val correctAnswer = WithCTE( + OneRowRelation() + .select( + extractorExpression(id, analyzedMergedSubquery.output, 0), + extractorExpression(id, analyzedMergedSubquery.output, 1), + extractorExpression(id, analyzedMergedSubquery.output, 2) + ), + Seq(definitionNode(analyzedMergedSubquery, id)) + ) + // Check the plan applied MergePaimonScalarSubqueries. + comparePlans(optimizedPlan.analyze, correctAnswer.analyze) + + // Check the query's result. + checkDataset(query.as[(Long, Long, Double)], (3L, 6L, 22.2)) + } + } + + test("Paimon Optimization: paimon scan equals") { + withTable("T") { + spark.sql(s"CREATE TABLE T (id INT, name STRING, pt STRING) PARTITIONED BY (pt)") + spark.sql(s"INSERT INTO T VALUES (1, 'a', 'p1'), (2, 'b', 'p1'), (3, 'c', 'p2')") + + // data filter and partition filter + val sqlText = "SELECT * FROM T WHERE id = 1 AND pt = 'p1' LIMIT 1" + Assertions.assertEquals(getPaimonScan(sqlText), getPaimonScan(sqlText)) + + // topN + val sqlText2 = "SELECT id FROM T ORDER BY id ASC NULLS LAST LIMIT 5" + Assertions.assertEquals(getPaimonScan(sqlText2), getPaimonScan(sqlText2)) + } + } + + test(s"Paimon Optimization: optimize metadata only delete") { + for (useV2Write <- Seq("false")) { + withSparkSQLConf("spark.paimon.write.use-v2-write" -> useV2Write) { + withTable("t") { + sql(s""" + |CREATE TABLE t (id INT, name STRING, pt INT) + |PARTITIONED BY (pt) + |""".stripMargin) + sql("INSERT INTO t VALUES (1, 'a', 1), (2, 'b', 2)") + val df = sql("DELETE FROM t WHERE pt = 1") + checkTruncatePaimonTable(df) + checkAnswer(sql("SELECT * FROM t ORDER BY id"), Seq(Row(2, "b", 2))) + } + } + } + } + + test(s"Paimon Optimization: eval subqueries for delete table with ScalarSubquery") { + withPk.foreach( + hasPk => { + val tblProps = if (hasPk) { + s"TBLPROPERTIES ('primary-key'='id, pt')" + } else { + "" + } + withTable("t1", "t2") { + spark.sql(s""" + |CREATE TABLE t1 (id INT, name STRING, pt INT) + |$tblProps + |PARTITIONED BY (pt) + |""".stripMargin) + spark.sql( + "INSERT INTO t1 VALUES (1, 'a', 1), (2, 'b', 2), (3, 'c', 2), (4, 'd', 3), (5, 'e', 4)") + + spark.sql(s"CREATE TABLE t2 (id INT, n INT)") + spark.sql("INSERT INTO t2 VALUES (1, 1), (2, 2), (3, 3), (4, 4)") + + val df = + spark.sql(s"""DELETE FROM t1 WHERE + |pt >= (SELECT min(id) FROM t2 WHERE n BETWEEN 2 AND 3) + |AND + |pt <= (SELECT max(id) FROM t2 WHERE n BETWEEN 2 AND 3)""".stripMargin) + // For partition-only predicates, drop partition is called internally. + Assertions.assertEquals( + CommitKind.OVERWRITE, + loadTable("t1").store().snapshotManager().latestSnapshot().commitKind()) + checkTruncatePaimonTable(df) + + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER BY id"), + Row(1, "a", 1) :: Row(5, "e", 4) :: Nil) + + // subquery eval nothing + spark.sql(s"""DELETE FROM t1 WHERE + |pt >= (SELECT min(id) FROM t2 WHERE n > 10)""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER BY id"), + Row(1, "a", 1) :: Row(5, "e", 4) :: Nil) + } + }) + } + + test(s"Paimon Optimization: eval subqueries for delete table with InSubquery") { + withPk.foreach( + hasPk => { + val tblProps = if (hasPk) { + s"TBLPROPERTIES ('primary-key'='id, pt')" + } else { + "" + } + withTable("t1", "t2") { + spark.sql(s""" + |CREATE TABLE t1 (id INT, name STRING, pt INT) + |$tblProps + |PARTITIONED BY (pt) + |""".stripMargin) + spark.sql( + "INSERT INTO t1 VALUES (1, 'a', 1), (2, 'b', 2), (3, 'c', 2), (4, 'd', 3), (5, 'e', 4)") + + spark.sql(s"CREATE TABLE t2 (id INT, n INT)") + spark.sql("INSERT INTO t2 VALUES (1, 1), (2, 2), (3, 3), (4, 4)") + + val df = + spark.sql(s"""DELETE FROM t1 WHERE + |pt in (SELECT id FROM t2 WHERE n BETWEEN 2 AND 3) + |OR + |pt in (SELECT max(id) FROM t2 WHERE n BETWEEN 2 AND 3)""".stripMargin) + // For partition-only predicates, drop partition is called internally. + Assertions.assertEquals( + CommitKind.OVERWRITE, + loadTable("t1").store().snapshotManager().latestSnapshot().commitKind()) + checkTruncatePaimonTable(df) + + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER BY id"), + Row(1, "a", 1) :: Row(5, "e", 4) :: Nil) + + // subquery eval nothing + spark.sql(s"""DELETE FROM t1 WHERE + |pt in (SELECT id FROM t2 WHERE n > 10)""".stripMargin) + + checkAnswer( + spark.sql("SELECT * FROM t1 ORDER BY id"), + Row(1, "a", 1) :: Row(5, "e", 4) :: Nil) + } + }) + } + + private def checkTruncatePaimonTable(df: DataFrame): Unit = { + val plan = df.queryExecution.executedPlan.asInstanceOf[CommandResultExec].commandPhysicalPlan + assert(plan.isInstanceOf[TruncatePaimonTableWithFilterExec]) + } +} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala new file mode 100644 index 000000000000..26677d85c71a --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonPushDownTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class PaimonPushDownTest extends PaimonPushDownTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTest.scala new file mode 100644 index 000000000000..f37fbad27033 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonV1FunctionTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class PaimonV1FunctionTest extends PaimonV1FunctionTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala new file mode 100644 index 000000000000..6ab8a2671b51 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/PaimonViewTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class PaimonViewTest extends PaimonViewTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RewriteUpsertTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RewriteUpsertTableTest.scala new file mode 100644 index 000000000000..412aa3b30351 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RewriteUpsertTableTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class RewriteUpsertTableTest extends RewriteUpsertTableTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowIdPushDownTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowIdPushDownTest.scala new file mode 100644 index 000000000000..da4c9b854df3 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowIdPushDownTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class RowIdPushDownTest extends RowIdPushDownTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala new file mode 100644 index 000000000000..9f96840a7788 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/RowTrackingTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class RowTrackingTest extends RowTrackingTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.scala new file mode 100644 index 000000000000..6601dc2fca37 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/ShowColumnsTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class ShowColumnsTest extends PaimonShowColumnsTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/SparkV2FilterConverterTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/SparkV2FilterConverterTest.scala new file mode 100644 index 000000000000..21c4c8a495ed --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/SparkV2FilterConverterTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class SparkV2FilterConverterTest extends SparkV2FilterConverterTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala new file mode 100644 index 000000000000..92309d54167b --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/TagDdlTest.scala @@ -0,0 +1,21 @@ +/* + * 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.paimon.spark.sql + +class TagDdlTest extends PaimonTagDdlTestBase {} diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.scala new file mode 100644 index 000000000000..0a56fa7eced1 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/UpdateTableTest.scala @@ -0,0 +1,32 @@ +/* + * 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.paimon.spark.sql + +import org.apache.spark.SparkConf + +class UpdateTableTest extends UpdateTableTestBase { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.write.use-v2-write", "false") + } +} + +// V2 row-level operations (delete/update/merge) are not supported in Spark 4.1 because +// RewriteDeleteFromTable/RewriteUpdateTable moved into the Resolution batch and runs before +// Paimon's post-hoc rules. SparkTable does not implement SupportsRowLevelOperations in the 4.1 +// shim to avoid this conflict. diff --git a/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/VariantTest.scala b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/VariantTest.scala new file mode 100644 index 000000000000..94e9ac683f02 --- /dev/null +++ b/paimon-spark/paimon-spark-4.1/src/test/scala/org/apache/paimon/spark/sql/VariantTest.scala @@ -0,0 +1,33 @@ +/* + * 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.paimon.spark.sql + +import org.apache.spark.SparkConf + +class VariantTest extends VariantTestBase { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.variant.inferShreddingSchema", "false") + } +} + +class VariantInferShreddingTest extends VariantTestBase { + override protected def sparkConf: SparkConf = { + super.sparkConf.set("spark.paimon.variant.inferShreddingSchema", "true") + } +} diff --git a/pom.xml b/pom.xml index 5a329e76f6cf..fd6666f94c74 100644 --- a/pom.xml +++ b/pom.xml @@ -92,7 +92,7 @@ under the License. 1.20.1 2.12 2.12.18 - 2.13.16 + 2.13.17 ${scala212.version} ${scala212.version} 1.1.10.8 @@ -427,6 +427,7 @@ under the License. paimon-spark/paimon-spark4-common paimon-spark/paimon-spark-4.0 + paimon-spark/paimon-spark-4.1 17