-
Notifications
You must be signed in to change notification settings - Fork 2.5k
[HUDI-2811] Support Spark 3.2 #4270
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
072c229
4d7d4d0
d3e4623
863dd41
cbf2686
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -18,18 +18,30 @@ | |
| package org.apache.spark.sql.avro | ||
|
|
||
| import org.apache.avro.Schema | ||
|
|
||
| import org.apache.hudi.HoodieSparkUtils | ||
|
|
||
| import org.apache.spark.sql.types.DataType | ||
|
|
||
| /** | ||
| * This is to be compatible with the type returned by Spark 3.1 | ||
| * and other spark versions for AvroDeserializer | ||
| */ | ||
| case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) | ||
| extends AvroDeserializer(rootAvroType, rootCatalystType) { | ||
| case class HoodieAvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SPARK-34404 |
||
|
|
||
| private val avroDeserializer = if (HoodieSparkUtils.isSpark3_2) { | ||
| // SPARK-34404: As of Spark3.2, there is no AvroDeserializer's constructor with Schema and DataType arguments. | ||
| // So use the reflection to get AvroDeserializer instance. | ||
| val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType], classOf[String]) | ||
| constructor.newInstance(rootAvroType, rootCatalystType, "EXCEPTION") | ||
| } else { | ||
| val constructor = classOf[AvroDeserializer].getConstructor(classOf[Schema], classOf[DataType]) | ||
| constructor.newInstance(rootAvroType, rootCatalystType) | ||
| } | ||
|
|
||
| def deserializeData(data: Any): Any = { | ||
| super.deserialize(data) match { | ||
| case Some(r) => r // spark 3.1 return type is Option, we fetch the data. | ||
| avroDeserializer.deserialize(data) match { | ||
| case Some(r) => r // As of spark 3.1, this will return data wrapped with Option, so we fetch the data. | ||
| case o => o // for other spark version, return the data directly. | ||
| } | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -22,17 +22,37 @@ import org.apache.spark.sql.catalyst.plans.logical.CompactionOperation.Compactio | |
| case class CompactionTable(table: LogicalPlan, operation: CompactionOperation, instantTimestamp: Option[Long]) | ||
| extends Command { | ||
| override def children: Seq[LogicalPlan] = Seq(table) | ||
|
|
||
| def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionTable = { | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SPARK-34989
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. can we link the spark jira link in the codebase to help users understand the background |
||
| copy(table = newChildren.head) | ||
| } | ||
| } | ||
|
|
||
| case class CompactionPath(path: String, operation: CompactionOperation, instantTimestamp: Option[Long]) | ||
| extends Command | ||
| extends Command { | ||
| override def children: Seq[LogicalPlan] = Seq.empty | ||
|
|
||
| def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionPath = { | ||
| this | ||
| } | ||
| } | ||
|
|
||
| case class CompactionShowOnTable(table: LogicalPlan, limit: Int = 20) | ||
| extends Command { | ||
| override def children: Seq[LogicalPlan] = Seq(table) | ||
|
|
||
| def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionShowOnTable = { | ||
| copy(table = newChildren.head) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. would you please clarify why here use copy while the other just use this?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Other classes which extends
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @YannByron thanks for the explanation; looks like worth noting down as block comment there. |
||
| } | ||
| } | ||
|
|
||
| case class CompactionShowOnPath(path: String, limit: Int = 20) extends Command | ||
| case class CompactionShowOnPath(path: String, limit: Int = 20) extends Command { | ||
| override def children: Seq[LogicalPlan] = Seq.empty | ||
|
|
||
| def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]): CompactionShowOnPath = { | ||
| this | ||
| } | ||
| } | ||
|
|
||
| object CompactionOperation extends Enumeration { | ||
| type CompactionOperation = Value | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,30 @@ | ||
| /* | ||
| * 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.trees | ||
|
|
||
| /** | ||
| * Similar to `LeafLike` in Spark3.2. | ||
| */ | ||
| trait HoodieLeafLike[T <: TreeNode[T]] { self: TreeNode[T] => | ||
|
|
||
| override final def children: Seq[T] = Nil | ||
|
|
||
| override final def mapChildren(f: T => T): T = this.asInstanceOf[T] | ||
|
|
||
| final def withNewChildrenInternal(newChildren: IndexedSeq[T]): T = this.asInstanceOf[T] | ||
| } |
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -17,12 +17,13 @@ | |
|
|
||
| package org.apache.spark.sql.hudi.analysis | ||
|
|
||
| import org.apache.hudi.{HoodieSparkUtils, SparkAdapterSupport} | ||
| import org.apache.hudi.DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL | ||
| import org.apache.hudi.SparkAdapterSupport | ||
| import org.apache.hudi.common.model.HoodieRecord | ||
| import org.apache.hudi.common.table.HoodieTableMetaClient | ||
|
|
||
| import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedStar} | ||
| import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, Literal, NamedExpression} | ||
| import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, Expression, Literal, NamedExpression} | ||
| import org.apache.spark.sql.catalyst.plans.Inner | ||
| import org.apache.spark.sql.catalyst.plans.logical._ | ||
| import org.apache.spark.sql.catalyst.rules.Rule | ||
|
|
@@ -137,7 +138,7 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi | |
| // We can do this because under the normal case, we should not allow to update or set | ||
| // the hoodie's meta field in sql statement, it is a system field, cannot set the value | ||
| // by user. | ||
| if (HoodieSqlUtils.isSpark3) { | ||
| if (HoodieSparkUtils.isSpark3) { | ||
| val assignmentFieldNames = assignments.map(_.key).map { | ||
| case attr: AttributeReference => | ||
| attr.name | ||
|
|
@@ -178,11 +179,19 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi | |
| .map { case (targetAttr, sourceAttr) => Assignment(targetAttr, sourceAttr) } | ||
| } | ||
| } else { | ||
| assignments.map(assignment => { | ||
| // For Spark3.2, InsertStarAction/UpdateStarAction's assignments will contain the meta fields. | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. so here meta fields is hudi meta fields?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. yep. |
||
| val withoutMetaAttrs = assignments.filterNot{ assignment => | ||
| if (assignment.key.isInstanceOf[Attribute]) { | ||
| HoodieSqlUtils.isMetaField(assignment.key.asInstanceOf[Attribute].name) | ||
| } else { | ||
| false | ||
| } | ||
| } | ||
| withoutMetaAttrs.map { assignment => | ||
| val resolvedKey = resolveExpressionFrom(target)(assignment.key) | ||
| val resolvedValue = resolveExpressionFrom(resolvedSource, Some(target))(assignment.value) | ||
| Assignment(resolvedKey, resolvedValue) | ||
| }) | ||
| } | ||
| } | ||
| (resolvedCondition, resolvedAssignments) | ||
| } | ||
|
|
@@ -242,13 +251,21 @@ case class HoodieResolveReferences(sparkSession: SparkSession) extends Rule[Logi | |
| case DeleteAction(condition) => | ||
| val resolvedCondition = condition.map(resolveExpressionFrom(resolvedSource)(_)) | ||
| DeleteAction(resolvedCondition) | ||
| case action: MergeAction => | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SPARK-34962
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
ditto, link the spark jira link |
||
| // SPARK-34962: use UpdateStarAction as the explicit representation of * in UpdateAction. | ||
| // So match and covert this in Spark3.2 env. | ||
| UpdateAction(action.condition, Seq.empty) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the UpdateAction is not aligned with the description?
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
| } | ||
| // Resolve the notMatchedActions | ||
| val resolvedNotMatchedActions = notMatchedActions.map { | ||
| case InsertAction(condition, assignments) => | ||
| val (resolvedCondition, resolvedAssignments) = | ||
| resolveConditionAssignments(condition, assignments) | ||
| InsertAction(resolvedCondition, resolvedAssignments) | ||
| case action: MergeAction => | ||
| // SPARK-34962: use InsertStarAction as the explicit representation of * in InsertAction. | ||
| // So match and covert this in Spark3.2 env. | ||
| InsertAction(action.condition, Seq.empty) | ||
| } | ||
| // Return the resolved MergeIntoTable | ||
| MergeIntoTable(target, resolvedSource, resolvedMergeCondition, | ||
|
|
@@ -426,9 +443,11 @@ case class HoodiePostAnalysisRule(sparkSession: SparkSession) extends Rule[Logic | |
| case AlterTableChangeColumnCommand(tableName, columnName, newColumn) | ||
| if isHoodieTable(tableName, sparkSession) => | ||
| AlterHoodieTableChangeColumnCommand(tableName, columnName, newColumn) | ||
| case ShowPartitionsCommand(tableName, specOpt) | ||
| if isHoodieTable(tableName, sparkSession) => | ||
| ShowHoodieTablePartitionsCommand(tableName, specOpt) | ||
| // SPARK-34238: the definition of ShowPartitionsCommand has been changed in Spark3.2. | ||
| // Match the class type instead of call the `unapply` method. | ||
| case s: ShowPartitionsCommand | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. SPARK-34238 |
||
| if isHoodieTable(s.tableName, sparkSession) => | ||
| ShowHoodieTablePartitionsCommand(s.tableName, s.spec) | ||
| // Rewrite TruncateTableCommand to TruncateHoodieTableCommand | ||
| case TruncateTableCommand(tableName, partitionSpec) | ||
| if isHoodieTable(tableName, sparkSession) => | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,7 +31,7 @@ import org.apache.spark.api.java.JavaSparkContext | |
| import org.apache.spark.sql.{AnalysisException, Row, SparkSession} | ||
| import org.apache.spark.sql.catalyst.TableIdentifier | ||
| import org.apache.spark.sql.catalyst.catalog.{CatalogTable, HoodieCatalogTable} | ||
| import org.apache.spark.sql.execution.command.{DDLUtils, RunnableCommand} | ||
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.types.{StructField, StructType} | ||
| import org.apache.spark.sql.util.SchemaUtils | ||
|
|
||
|
|
@@ -44,7 +44,7 @@ import scala.util.control.NonFatal | |
| case class AlterHoodieTableAddColumnsCommand( | ||
| tableId: TableIdentifier, | ||
| colsToAdd: Seq[StructField]) | ||
| extends RunnableCommand { | ||
| extends HoodieLeafRunnableCommand { | ||
|
|
||
| override def run(sparkSession: SparkSession): Seq[Row] = { | ||
| if (colsToAdd.nonEmpty) { | ||
|
|
@@ -74,7 +74,7 @@ case class AlterHoodieTableAddColumnsCommand( | |
| } | ||
|
|
||
| private def refreshSchemaInMeta(sparkSession: SparkSession, table: CatalogTable, | ||
| newSqlSchema: StructType): Unit = { | ||
| newSqlDataSchema: StructType): Unit = { | ||
| try { | ||
| sparkSession.catalog.uncacheTable(tableId.quotedString) | ||
| } catch { | ||
|
|
@@ -84,12 +84,11 @@ case class AlterHoodieTableAddColumnsCommand( | |
| sparkSession.catalog.refreshTable(table.identifier.unquotedString) | ||
|
|
||
| SchemaUtils.checkColumnNameDuplication( | ||
| newSqlSchema.map(_.name), | ||
| newSqlDataSchema.map(_.name), | ||
| "in the table definition of " + table.identifier, | ||
| conf.caseSensitiveAnalysis) | ||
| DDLUtils.checkDataColNames(table, colsToAdd.map(_.name)) | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. the definition of |
||
|
|
||
| sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlSchema) | ||
| sparkSession.sessionState.catalog.alterTableDataSchema(tableId, newSqlDataSchema) | ||
| } | ||
| } | ||
|
|
||
|
|
||
Uh oh!
There was an error while loading. Please reload this page.