1
0

[HUDI-3338] Custom relation instead of HadoopFsRelation (#4709)

Currently, HadoopFsRelation will use the value of the real partition path as the value of the partition field. However, different from the normal table, Hudi will persist the partition value in the parquet file. And in some cases, it's different between the value of the real partition path and the value of the partition field.
So here we implement BaseFileOnlyViewRelation which lets Hudi manage its own relation.
This commit is contained in:
Yann Byron
2022-02-12 02:48:44 +08:00
committed by GitHub
parent 10474e0962
commit b431246710
20 changed files with 685 additions and 235 deletions

View File

@@ -19,6 +19,7 @@ package org.apache.spark.sql.adapter
import org.apache.hudi.Spark2RowSerDe
import org.apache.hudi.client.utils.SparkRowSerDe
import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.catalyst.expressions.{Expression, Like}
@@ -26,12 +27,14 @@ 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.{AliasIdentifier, TableIdentifier}
import org.apache.spark.sql.execution.datasources.{Spark2ParsePartitionUtil, SparkParsePartitionUtil}
import org.apache.spark.sql.execution.datasources.{FilePartition, 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.{Row, SparkSession}
import scala.collection.mutable.ArrayBuffer
/**
* The adapter for spark2.
*/
@@ -86,4 +89,44 @@ class Spark2Adapter extends SparkAdapter {
override def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String] = {
throw new IllegalStateException(s"Should not call ParserInterface#parseMultipartIdentifier for spark2")
}
/**
* Combine [[PartitionedFile]] to [[FilePartition]] according to `maxSplitBytes`.
*
* This is a copy of org.apache.spark.sql.execution.datasources.FilePartition#getFilePartitions from Spark 3.2.
* And this will be called only in Spark 2.
*/
override def getFilePartitions(
sparkSession: SparkSession,
partitionedFiles: Seq[PartitionedFile],
maxSplitBytes: Long): Seq[FilePartition] = {
val partitions = new ArrayBuffer[FilePartition]
val currentFiles = new ArrayBuffer[PartitionedFile]
var currentSize = 0L
/** Close the current partition and move to the next. */
def closePartition(): Unit = {
if (currentFiles.nonEmpty) {
// Copy to a new Array.
val newPartition = FilePartition(partitions.size, currentFiles.toArray)
partitions += newPartition
}
currentFiles.clear()
currentSize = 0
}
val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes
// Assign files to partitions using "Next Fit Decreasing"
partitionedFiles.foreach { file =>
if (currentSize + file.length > maxSplitBytes) {
closePartition()
}
// Add the given file to the current partition.
currentSize += file.length + openCostInBytes
currentFiles += file
}
closePartition()
partitions.toSeq
}
}