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

@@ -52,6 +52,8 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
globPaths: Seq[Path])
extends HoodieBaseRelation(sqlContext, metaClient, optParams, userSchema) with SparkAdapterSupport {
case class HoodieBaseFileSplit(filePartition: FilePartition) extends HoodieFileSplit
override type FileSplit = HoodieBaseFileSplit
// TODO(HUDI-3204) this is to override behavior (exclusively) for COW tables to always extract
@@ -97,7 +99,9 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
// back into the one expected by the caller
val projectedReader = projectReader(baseFileReader, requiredSchema.structTypeSchema)
new HoodieFileScanRDD(sparkSession, projectedReader.apply, fileSplits)
// SPARK-37273 FileScanRDD constructor changed in SPARK 3.3
sparkAdapter.createHoodieFileScanRDD(sparkSession, projectedReader.apply, fileSplits.map(_.filePartition), requiredSchema.structTypeSchema)
.asInstanceOf[HoodieUnsafeRDD]
}
protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[HoodieBaseFileSplit] = {

View File

@@ -1,34 +0,0 @@
/*
* 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.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.{FilePartition, FileScanRDD, PartitionedFile}
case class HoodieBaseFileSplit(filePartition: FilePartition) extends HoodieFileSplit
class HoodieFileScanRDD(@transient private val sparkSession: SparkSession,
read: PartitionedFile => Iterator[InternalRow],
@transient fileSplits: Seq[HoodieBaseFileSplit])
extends FileScanRDD(sparkSession, read, fileSplits.map(_.filePartition))
with HoodieUnsafeRDD {
override final def collect(): Array[InternalRow] = super[HoodieUnsafeRDD].collect()
}