1
0

[HUDI-4186] Support Hudi with Spark 3.3.0 (#5943)

Co-authored-by: Shawn Chang <yxchang@amazon.com>
This commit is contained in:
Shawn Chang
2022-07-27 14:47:49 -07:00
committed by GitHub
parent 924c30c7ea
commit cdaec5a8da
99 changed files with 10865 additions and 104 deletions

View File

@@ -0,0 +1,35 @@
/*
* 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.hudi
import org.apache.hudi.HoodieUnsafeRDD
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.AttributeReference
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
import org.apache.spark.sql.types.StructType
class Spark2HoodieFileScanRDD(@transient private val sparkSession: SparkSession,
read: PartitionedFile => Iterator[InternalRow],
@transient filePartitions: Seq[FilePartition])
extends FileScanRDD(sparkSession, read, filePartitions)
with HoodieUnsafeRDD {
override final def collect(): Array[InternalRow] = super[HoodieUnsafeRDD].collect()
}

View File

@@ -19,22 +19,23 @@
package org.apache.spark.sql.adapter
import org.apache.avro.Schema
import org.apache.hudi.Spark2RowSerDe
import org.apache.hudi.{Spark2HoodieFileScanRDD, Spark2RowSerDe}
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.avro._
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, InterpretedPredicate}
import org.apache.spark.sql.catalyst.parser.ParserInterface
import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, Join, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.logical.{Command, InsertIntoTable, Join, LogicalPlan, DeleteFromTable}
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, Spark24HoodieParquetFileFormat}
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile, Spark2ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.hudi.SparkAdapter
import org.apache.spark.sql.hudi.parser.HoodieSpark2ExtendedSqlParser
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.sql.types.{DataType, StructType}
import org.apache.spark.sql.{HoodieCatalystExpressionUtils, HoodieCatalystPlansUtils, HoodieSpark2CatalystExpressionUtils, HoodieSpark2CatalystPlanUtils, Row, SparkSession}
import org.apache.spark.storage.StorageLevel
import org.apache.spark.storage.StorageLevel._
@@ -122,6 +123,30 @@ class Spark2Adapter extends SparkAdapter {
InterpretedPredicate.create(e)
}
override def createHoodieFileScanRDD(sparkSession: SparkSession,
readFunction: PartitionedFile => Iterator[InternalRow],
filePartitions: Seq[FilePartition],
readDataSchema: StructType,
metadataColumns: Seq[AttributeReference] = Seq.empty): FileScanRDD = {
new Spark2HoodieFileScanRDD(sparkSession, readFunction, filePartitions)
}
override def resolveDeleteFromTable(deleteFromTable: Command,
resolveExpression: Expression => Expression): DeleteFromTable = {
val deleteFromTableCommand = deleteFromTable.asInstanceOf[DeleteFromTable]
val resolvedCondition = deleteFromTableCommand.condition.map(resolveExpression)
DeleteFromTable(deleteFromTableCommand.table, resolvedCondition)
}
override def extractCondition(deleteFromTable: Command): Expression = {
deleteFromTable.asInstanceOf[DeleteFromTable].condition.getOrElse(null)
}
override def getQueryParserFromExtendedSqlParser(session: SparkSession, delegate: ParserInterface,
sqlText: String): LogicalPlan = {
throw new UnsupportedOperationException(s"Unsupported parseQuery method in Spark earlier than Spark 3.3.0")
}
override def convertStorageLevelToString(level: StorageLevel): String = level match {
case NONE => "NONE"
case DISK_ONLY => "DISK_ONLY"