[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:
@@ -117,7 +117,7 @@ object AvroConversionUtils {
|
||||
|
||||
def buildAvroRecordBySchema(record: IndexedRecord,
|
||||
requiredSchema: Schema,
|
||||
requiredPos: List[Int],
|
||||
requiredPos: Seq[Int],
|
||||
recordBuilder: GenericRecordBuilder): GenericRecord = {
|
||||
val requiredFields = requiredSchema.getFields.asScala
|
||||
assert(requiredFields.length == requiredPos.length)
|
||||
|
||||
@@ -38,11 +38,11 @@ import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql.catalyst.encoders.RowEncoder
|
||||
import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal}
|
||||
import org.apache.spark.sql.execution.datasources.{FileStatusCache, InMemoryFileIndex}
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.sources._
|
||||
import org.apache.spark.sql.types.{StringType, StructField, StructType}
|
||||
import org.apache.spark.sql.{DataFrame, SparkSession}
|
||||
|
||||
import scala.collection.JavaConverters._
|
||||
import scala.collection.JavaConverters.asScalaBufferConverter
|
||||
|
||||
object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
@@ -293,4 +293,30 @@ object HoodieSparkUtils extends SparkAdapterSupport {
|
||||
s"${tableSchema.fieldNames.mkString(",")}")
|
||||
AttributeReference(columnName, field.get.dataType, field.get.nullable)()
|
||||
}
|
||||
|
||||
def getRequiredSchema(tableAvroSchema: Schema, requiredColumns: Array[String]): (Schema, StructType) = {
|
||||
// First get the required avro-schema, then convert the avro-schema to spark schema.
|
||||
val name2Fields = tableAvroSchema.getFields.asScala.map(f => f.name() -> f).toMap
|
||||
// Here have to create a new Schema.Field object
|
||||
// to prevent throwing exceptions like "org.apache.avro.AvroRuntimeException: Field already used".
|
||||
val requiredFields = requiredColumns.map(c => name2Fields(c))
|
||||
.map(f => new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())).toList
|
||||
val requiredAvroSchema = Schema.createRecord(tableAvroSchema.getName, tableAvroSchema.getDoc,
|
||||
tableAvroSchema.getNamespace, tableAvroSchema.isError, requiredFields.asJava)
|
||||
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
|
||||
(requiredAvroSchema, requiredStructSchema)
|
||||
}
|
||||
|
||||
def toAttribute(tableSchema: StructType): Seq[AttributeReference] = {
|
||||
tableSchema.map { field =>
|
||||
AttributeReference(field.name, field.dataType, field.nullable, field.metadata)()
|
||||
}
|
||||
}
|
||||
|
||||
def collectFieldIndexes(projectedSchema: StructType, originalSchema: StructType): Seq[Int] = {
|
||||
val nameToIndex = originalSchema.fields.zipWithIndex.map{ case (field, index) =>
|
||||
field.name -> index
|
||||
}.toMap
|
||||
projectedSchema.map(field => nameToIndex(field.name))
|
||||
}
|
||||
}
|
||||
|
||||
@@ -19,6 +19,7 @@
|
||||
package org.apache.spark.sql.hudi
|
||||
|
||||
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
|
||||
@@ -26,7 +27,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface
|
||||
import org.apache.spark.sql.catalyst.plans.JoinType
|
||||
import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan}
|
||||
import org.apache.spark.sql.catalyst.{AliasIdentifier, TableIdentifier}
|
||||
import org.apache.spark.sql.execution.datasources.SparkParsePartitionUtil
|
||||
import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, SparkParsePartitionUtil}
|
||||
import org.apache.spark.sql.internal.SQLConf
|
||||
import org.apache.spark.sql.{Row, SparkSession}
|
||||
|
||||
@@ -92,4 +93,10 @@ trait SparkAdapter extends Serializable {
|
||||
* ParserInterface#parseMultipartIdentifier is supported since spark3, for spark2 this should not be called.
|
||||
*/
|
||||
def parseMultipartIdentifier(parser: ParserInterface, sqlText: String): Seq[String]
|
||||
|
||||
/**
|
||||
* Combine [[PartitionedFile]] to [[FilePartition]] according to `maxSplitBytes`.
|
||||
*/
|
||||
def getFilePartitions(sparkSession: SparkSession, partitionedFiles: Seq[PartitionedFile],
|
||||
maxSplitBytes: Long): Seq[FilePartition]
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user