1
0

[HUDI-3204] Fixing partition-values being derived from partition-path instead of source columns (#5364)

- Scaffolded `Spark24HoodieParquetFileFormat` extending `ParquetFileFormat` and overriding the behavior of adding partition columns to every row
 - Amended `SparkAdapter`s `createHoodieParquetFileFormat` API to be able to configure whether to append partition values or not
 - Fallback to append partition values in cases when the source columns are not persisted in data-file
 - Fixing HoodieBaseRelation incorrectly handling mandatory columns
This commit is contained in:
Alexey Kudinkin
2022-04-20 04:30:27 -07:00
committed by GitHub
parent 408663c42b
commit f7544e23ac
28 changed files with 1156 additions and 686 deletions

View File

@@ -17,4 +17,4 @@
org.apache.hudi.DefaultSource
org.apache.spark.sql.execution.datasources.parquet.SparkHoodieParquetFileFormat
org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat

View File

@@ -20,14 +20,13 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
import org.apache.hudi.common.model.HoodieFileFormat
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.hadoop.HoodieROTablePathFilter
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources._
import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat
import org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetFileFormat, ParquetFileFormat}
import org.apache.spark.sql.hive.orc.OrcFileFormat
import org.apache.spark.sql.sources.{BaseRelation, Filter}
import org.apache.spark.sql.types.StructType
@@ -56,6 +55,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
override type FileSplit = HoodieBaseFileSplit
override lazy val mandatoryColumns: Seq[String] =
// TODO reconcile, record's key shouldn't be mandatory for base-file only relation
Seq(recordKeyField)
override def imbueConfigs(sqlContext: SQLContext): Unit = {
@@ -65,14 +65,14 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
protected override def composeRDD(fileSplits: Seq[HoodieBaseFileSplit],
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Array[Filter]): HoodieUnsafeRDD = {
val baseFileReader = createBaseFileReader(
spark = sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = requiredSchema,
filters = filters,
options = optParams,
@@ -114,16 +114,38 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
* rule; you can find more details in HUDI-3896)
*/
def toHadoopFsRelation: HadoopFsRelation = {
// We're delegating to Spark to append partition values to every row only in cases
// when these corresponding partition-values are not persisted w/in the data file itself
val shouldAppendPartitionColumns = shouldOmitPartitionColumns
val (tableFileFormat, formatClassName) = metaClient.getTableConfig.getBaseFileFormat match {
case HoodieFileFormat.PARQUET => (new ParquetFileFormat, "parquet")
case HoodieFileFormat.PARQUET =>
(sparkAdapter.createHoodieParquetFileFormat(shouldAppendPartitionColumns).get, HoodieParquetFileFormat.FILE_FORMAT_ID)
case HoodieFileFormat.ORC => (new OrcFileFormat, "orc")
}
if (globPaths.isEmpty) {
// NOTE: There are currently 2 ways partition values could be fetched:
// - Source columns (producing the values used for physical partitioning) will be read
// from the data file
// - Values parsed from the actual partition pat would be appended to the final dataset
//
// In the former case, we don't need to provide the partition-schema to the relation,
// therefore we simply stub it w/ empty schema and use full table-schema as the one being
// read from the data file.
//
// In the latter, we have to specify proper partition schema as well as "data"-schema, essentially
// being a table-schema with all partition columns stripped out
val (partitionSchema, dataSchema) = if (shouldAppendPartitionColumns) {
(fileIndex.partitionSchema, fileIndex.dataSchema)
} else {
(StructType(Nil), tableStructSchema)
}
HadoopFsRelation(
location = fileIndex,
partitionSchema = fileIndex.partitionSchema,
dataSchema = fileIndex.dataSchema,
partitionSchema = partitionSchema,
dataSchema = dataSchema,
bucketSpec = None,
fileFormat = tableFileFormat,
optParams)(sparkSession)

View File

@@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path}
import org.apache.hadoop.hbase.io.hfile.CacheConfig
import org.apache.hadoop.mapred.JobConf
import org.apache.hudi.HoodieBaseRelation.getPartitionPath
import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, createHFileReader, generateUnsafeProjection, getPartitionPath}
import org.apache.hudi.HoodieConversionUtils.toScalaOption
import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration}
import org.apache.hudi.common.fs.FSUtils
@@ -36,12 +36,13 @@ import org.apache.hudi.common.util.ValidationUtils.checkState
import org.apache.hudi.internal.schema.InternalSchema
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
import org.apache.hudi.io.storage.HoodieHFileReader
import org.apache.spark.TaskContext
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.avro.HoodieAvroSchemaConverters
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression, UnsafeProjection}
import org.apache.spark.sql.execution.FileRelation
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile, PartitioningUtils}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
@@ -50,11 +51,11 @@ import org.apache.spark.sql.types.{StringType, StructField, StructType}
import org.apache.spark.sql.{Row, SQLContext, SparkSession}
import org.apache.spark.unsafe.types.UTF8String
import java.io.Closeable
import java.net.URI
import java.util.Locale
import scala.collection.JavaConverters._
import scala.util.Try
import scala.util.control.NonFatal
import scala.util.{Failure, Success, Try}
trait HoodieFileSplit {}
@@ -78,7 +79,6 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
extends BaseRelation
with FileRelation
with PrunedFilteredScan
with SparkAdapterSupport
with Logging {
type FileSplit <: HoodieFileSplit
@@ -125,14 +125,17 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema) = {
val schemaUtil = new TableSchemaResolver(metaClient)
val avroSchema = Try(schemaUtil.getTableAvroSchema).getOrElse(
// If there is no commit in the table, we can't get the schema
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
userSchema match {
case Some(s) => sparkAdapter.getAvroSchemaConverters.toAvroType(s, nullable = false, "record")
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
}
)
val avroSchema = Try(schemaUtil.getTableAvroSchema) match {
case Success(schema) => schema
case Failure(e) =>
logWarning("Failed to fetch schema from the table", e)
// If there is no commit in the table, we can't get the schema
// t/h [[TableSchemaResolver]], fallback to the provided [[userSchema]] instead.
userSchema match {
case Some(s) => convertToAvroSchema(s)
case _ => throw new IllegalArgumentException("User-provided schema is required in case the table is empty")
}
}
// try to find internalSchema
val internalSchemaFromMeta = try {
schemaUtil.getTableInternalSchemaFromCommitMetadata.orElse(InternalSchema.getEmptyInternalSchema)
@@ -146,11 +149,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
protected val partitionColumns: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty)
/**
* if true, need to deal with schema for creating file reader.
*/
protected val dropPartitionColumnsWhenWrite: Boolean =
metaClient.getTableConfig.isDropPartitionColumns && partitionColumns.nonEmpty
protected val shouldOmitPartitionColumns: Boolean =
metaClient.getTableConfig.shouldDropPartitionColumns && partitionColumns.nonEmpty
/**
* NOTE: PLEASE READ THIS CAREFULLY
@@ -205,14 +205,19 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
* NOTE: DO NOT OVERRIDE THIS METHOD
*/
override final def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = {
// NOTE: In case list of requested columns doesn't contain the Primary Key one, we
// NOTE: PLEAS READ CAREFULLY BEFORE MAKING CHANGES
//
// In case list of requested columns doesn't contain the Primary Key one, we
// have to add it explicitly so that
// - Merging could be performed correctly
// - In case 0 columns are to be fetched (for ex, when doing {@code count()} on Spark's [[Dataset]],
// Spark still fetches all the rows to execute the query correctly
// Spark still fetches all the rows to execute the query correctly
//
// It's okay to return columns that have not been requested by the caller, as those nevertheless will be
// filtered out upstream
// *Appending* additional columns to the ones requested by the caller is not a problem, as those
// will be "projected out" by the caller's projection;
//
// (!!!) IT'S CRITICAL TO AVOID REORDERING OF THE REQUESTED COLUMNS AS THIS WILL BREAK THE UPSTREAM
// PROJECTION
val fetchedColumns: Array[String] = appendMandatoryColumns(requiredColumns)
val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) =
@@ -223,56 +228,62 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
val fileSplits = collectFileSplits(partitionFilters, dataFilters)
val partitionSchema = if (dropPartitionColumnsWhenWrite) {
// when hoodie.datasource.write.drop.partition.columns is true, partition columns can't be persisted in
// data files.
StructType(partitionColumns.map(StructField(_, StringType)))
} else {
StructType(Nil)
}
val tableSchema = HoodieTableSchema(tableStructSchema, if (internalSchema.isEmptySchema) tableAvroSchema.toString else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString, internalSchema)
val dataSchema = if (dropPartitionColumnsWhenWrite) {
val dataStructType = StructType(tableStructSchema.filterNot(f => partitionColumns.contains(f.name)))
HoodieTableSchema(
dataStructType,
sparkAdapter.getAvroSchemaConverters.toAvroType(dataStructType, nullable = false, "record").toString()
)
} else {
tableSchema
}
val requiredSchema = if (dropPartitionColumnsWhenWrite) {
val requiredStructType = StructType(requiredStructSchema.filterNot(f => partitionColumns.contains(f.name)))
HoodieTableSchema(
requiredStructType,
sparkAdapter.getAvroSchemaConverters.toAvroType(requiredStructType, nullable = false, "record").toString()
)
} else {
HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
}
// Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]]
// Please check [[needConversion]] scala-doc for more details
if (fileSplits.nonEmpty)
composeRDD(fileSplits, partitionSchema, dataSchema, requiredSchema, filters).asInstanceOf[RDD[Row]]
else
val tableAvroSchemaStr =
if (internalSchema.isEmptySchema) tableAvroSchema.toString
else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchemaStr, internalSchema)
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
// Since schema requested by the caller might contain partition columns, we might need to
// prune it, removing all partition columns from it in case these columns are not persisted
// in the data files
//
// NOTE: This partition schema is only relevant to file reader to be able to embed
// values of partition columns (hereafter referred to as partition values) encoded into
// the partition path, and omitted from the data file, back into fetched rows;
// Note that, by default, partition columns are not omitted therefore specifying
// partition schema for reader is not required
val (partitionSchema, dataSchema, prunedRequiredSchema) =
tryPrunePartitionColumns(tableSchema, requiredSchema)
if (fileSplits.isEmpty) {
sparkSession.sparkContext.emptyRDD
} else {
val rdd = composeRDD(fileSplits, partitionSchema, dataSchema, prunedRequiredSchema, filters)
// NOTE: In case when partition columns have been pruned from the required schema, we have to project
// the rows from the pruned schema back into the one expected by the caller
val projectedRDD = if (prunedRequiredSchema.structTypeSchema != requiredSchema.structTypeSchema) {
rdd.mapPartitions { it =>
val fullPrunedSchema = StructType(prunedRequiredSchema.structTypeSchema.fields ++ partitionSchema.fields)
val unsafeProjection = generateUnsafeProjection(fullPrunedSchema, requiredSchema.structTypeSchema)
it.map(unsafeProjection)
}
} else {
rdd
}
// Here we rely on a type erasure, to workaround inherited API restriction and pass [[RDD[InternalRow]]] back as [[RDD[Row]]]
// Please check [[needConversion]] scala-doc for more details
projectedRDD.asInstanceOf[RDD[Row]]
}
}
/**
* Composes RDD provided file splits to read from, table and partition schemas, data filters to be applied
*
* @param fileSplits file splits to be handled by the RDD
* @param partitionSchema target table's partition schema
* @param tableSchema target table's schema
* @param dataSchema target table's data files' schema
* @param requiredSchema projected schema required by the reader
* @param filters data filters to be applied
* @return instance of RDD (implementing [[HoodieUnsafeRDD]])
*/
protected def composeRDD(fileSplits: Seq[FileSplit],
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Array[Filter]): HoodieUnsafeRDD
@@ -325,16 +336,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
}
protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
if (dropPartitionColumnsWhenWrite) {
if (requestedColumns.isEmpty) {
mandatoryColumns.toArray
} else {
requestedColumns
}
} else {
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
requestedColumns ++ missing
}
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
requestedColumns ++ missing
}
protected def getTableState: HoodieTableState = {
@@ -364,7 +367,7 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
protected def getPartitionColumnsAsInternalRow(file: FileStatus): InternalRow = {
try {
val tableConfig = metaClient.getTableConfig
if (dropPartitionColumnsWhenWrite) {
if (shouldOmitPartitionColumns) {
val relativePath = new URI(metaClient.getBasePath).relativize(new URI(file.getPath.getParent.toString)).toString
val hiveStylePartitioningEnabled = tableConfig.getHiveStylePartitioningEnable.toBoolean
if (hiveStylePartitioningEnabled) {
@@ -388,40 +391,47 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
InternalRow.empty
}
}
}
object HoodieBaseRelation {
def getPartitionPath(fileStatus: FileStatus): Path =
fileStatus.getPath.getParent
protected def getColName(f: StructField): String = {
if (sparkSession.sessionState.conf.caseSensitiveAnalysis) {
f.name
} else {
f.name.toLowerCase(Locale.ROOT)
}
}
/**
* Returns file-reader routine accepting [[PartitionedFile]] and returning an [[Iterator]]
* over [[InternalRow]]
*/
def createBaseFileReader(spark: SparkSession,
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
protected def createBaseFileReader(spark: SparkSession,
partitionSchema: StructType,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val hfileReader = createHFileReader(
spark = spark,
tableSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = requiredSchema,
filters = filters,
options = options,
hadoopConf = hadoopConf
)
// We're delegating to Spark to append partition values to every row only in cases
// when these corresponding partition-values are not persisted w/in the data file itself
val shouldAppendPartitionColumns = shouldOmitPartitionColumns
val parquetReader = HoodieDataSourceHelper.buildHoodieParquetReader(
sparkSession = spark,
dataSchema = tableSchema.structTypeSchema,
dataSchema = dataSchema.structTypeSchema,
partitionSchema = partitionSchema,
requiredSchema = requiredSchema.structTypeSchema,
filters = filters,
options = options,
hadoopConf = hadoopConf
hadoopConf = hadoopConf,
appendPartitionValues = shouldAppendPartitionColumns
)
partitionedFile => {
@@ -436,8 +446,38 @@ object HoodieBaseRelation {
}
}
private def tryPrunePartitionColumns(tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema): (StructType, HoodieTableSchema, HoodieTableSchema) = {
if (shouldOmitPartitionColumns) {
val partitionSchema = StructType(partitionColumns.map(StructField(_, StringType)))
val prunedDataStructSchema = prunePartitionColumns(tableSchema.structTypeSchema)
val prunedRequiredSchema = prunePartitionColumns(requiredSchema.structTypeSchema)
(partitionSchema,
HoodieTableSchema(prunedDataStructSchema, convertToAvroSchema(prunedDataStructSchema).toString),
HoodieTableSchema(prunedRequiredSchema, convertToAvroSchema(prunedRequiredSchema).toString))
} else {
(StructType(Nil), tableSchema, requiredSchema)
}
}
private def prunePartitionColumns(dataStructSchema: StructType): StructType =
StructType(dataStructSchema.filterNot(f => partitionColumns.contains(f.name)))
}
object HoodieBaseRelation extends SparkAdapterSupport {
private def generateUnsafeProjection(from: StructType, to: StructType) =
sparkAdapter.createCatalystExpressionUtils().generateUnsafeProjection(from, to)
def convertToAvroSchema(structSchema: StructType): Schema =
sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, "Record")
def getPartitionPath(fileStatus: FileStatus): Path =
fileStatus.getPath.getParent
private def createHFileReader(spark: SparkSession,
tableSchema: HoodieTableSchema,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],

View File

@@ -21,6 +21,7 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FileStatus
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
import org.apache.hudi.internal.schema.InternalSchema
import org.apache.hudi.internal.schema.utils.SerDeHelper
import org.apache.spark.sql.SparkSession
@@ -38,8 +39,8 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
/**
* Wrapper `buildReaderWithPartitionValues` of [[ParquetFileFormat]]
* to deal with [[ColumnarBatch]] when enable parquet vectorized reader if necessary.
* Wrapper for `buildReaderWithPartitionValues` of [[ParquetFileFormat]] handling [[ColumnarBatch]],
* when Parquet's Vectorized Reader is used
*/
def buildHoodieParquetReader(sparkSession: SparkSession,
dataSchema: StructType,
@@ -47,9 +48,11 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
hadoopConf: Configuration,
appendPartitionValues: Boolean = false): PartitionedFile => Iterator[InternalRow] = {
val readParquetFile: PartitionedFile => Iterator[Any] = sparkAdapter.createHoodieParquetFileFormat().get.buildReaderWithPartitionValues(
val parquetFileFormat: ParquetFileFormat = sparkAdapter.createHoodieParquetFileFormat(appendPartitionValues).get
val readParquetFile: PartitionedFile => Iterator[Any] = parquetFileFormat.buildReaderWithPartitionValues(
sparkSession = sparkSession,
dataSchema = dataSchema,
partitionSchema = partitionSchema,
@@ -91,9 +94,12 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
* @param validCommits valid commits, using give validCommits to validate all legal histroy Schema files, and return the latest one.
*/
def getConfigurationWithInternalSchema(conf: Configuration, internalSchema: InternalSchema, tablePath: String, validCommits: String): Configuration = {
conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tablePath)
conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
val querySchemaString = SerDeHelper.toJson(internalSchema)
if (!isNullOrEmpty(querySchemaString)) {
conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, tablePath)
conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
}
conf
}
}

View File

@@ -88,7 +88,7 @@ object HoodieSparkSqlWriter {
val (parameters, hoodieConfig) = mergeParamsAndGetHoodieConfig(optParams, tableConfig)
val originKeyGeneratorClassName = HoodieWriterUtils.getOriginKeyGenerator(parameters)
val timestampKeyGeneratorConfigs = extractConfigsRelatedToTimestmapBasedKeyGenerator(
val timestampKeyGeneratorConfigs = extractConfigsRelatedToTimestampBasedKeyGenerator(
originKeyGeneratorClassName, parameters)
//validate datasource and tableconfig keygen are the same
validateKeyGeneratorConfig(originKeyGeneratorClassName, tableConfig);
@@ -758,7 +758,7 @@ object HoodieSparkSqlWriter {
(params, HoodieWriterUtils.convertMapToHoodieConfig(params))
}
private def extractConfigsRelatedToTimestmapBasedKeyGenerator(keyGenerator: String,
private def extractConfigsRelatedToTimestampBasedKeyGenerator(keyGenerator: String,
params: Map[String, String]): Map[String, String] = {
if (keyGenerator.equals(classOf[TimestampBasedKeyGenerator].getCanonicalName) ||
keyGenerator.equals(classOf[TimestampBasedAvroKeyGenerator].getCanonicalName)) {

View File

@@ -20,8 +20,8 @@ package org.apache.hudi
import org.apache.avro.Schema
import org.apache.hudi.common.model.{HoodieCommitMetadata, HoodieFileFormat, HoodieRecord, HoodieReplaceCommitMetadata}
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import java.util.stream.Collectors
import java.util.stream.Collectors
import org.apache.hadoop.fs.{GlobPattern, Path}
import org.apache.hudi.client.common.HoodieSparkEngineContext
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
@@ -36,6 +36,7 @@ import org.apache.hudi.table.HoodieSparkTable
import org.apache.log4j.LogManager
import org.apache.spark.api.java.JavaSparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat
import org.apache.spark.sql.sources.{BaseRelation, TableScan}
import org.apache.spark.sql.types.StructType
import org.apache.spark.sql.{DataFrame, Row, SQLContext}
@@ -183,7 +184,7 @@ class IncrementalRelation(val sqlContext: SQLContext,
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath)
sqlContext.sparkContext.hadoopConfiguration.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
val formatClassName = metaClient.getTableConfig.getBaseFileFormat match {
case HoodieFileFormat.PARQUET => if (!internalSchema.isEmptySchema) "HoodieParquet" else "parquet"
case HoodieFileFormat.PARQUET => HoodieParquetFileFormat.FILE_FORMAT_ID
case HoodieFileFormat.ORC => "orc"
}
sqlContext.sparkContext.hadoopConfiguration.unset("mapreduce.input.pathFilter.class")

View File

@@ -19,9 +19,7 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{GlobPattern, Path}
import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
import org.apache.hudi.HoodieConversionUtils.toScalaOption
import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath
import org.apache.hudi.common.model.{FileSlice, HoodieRecord}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
@@ -61,14 +59,14 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
protected override def composeRDD(fileSplits: Seq[HoodieMergeOnReadFileSplit],
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Array[Filter]): HoodieMergeOnReadRDD = {
val fullSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = dataSchema,
// This file-reader is used to read base file records, subsequently merging them with the records
// stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding
// applying any user-defined filtering _before_ we complete combining them w/ delta-log records (to make sure that
@@ -86,7 +84,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
val requiredSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = requiredSchema,
filters = filters ++ incrementalSpanRecordFilters,
options = optParams,
@@ -99,7 +97,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
// TODO(HUDI-3639) implement incremental span record filtering w/in RDD to make sure returned iterator is appropriately
// filtered, since file-reader might not be capable to perform filtering
new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, requiredSchemaParquetReader,
tableSchema, requiredSchema, hoodieTableState, mergeType, fileSplits)
dataSchema, requiredSchema, hoodieTableState, mergeType, fileSplits)
}
override protected def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): List[HoodieMergeOnReadFileSplit] = {

View File

@@ -20,17 +20,14 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
import org.apache.hudi.HoodieBaseRelation.createBaseFileReader
import org.apache.hudi.HoodieConversionUtils.toScalaOption
import org.apache.hudi.MergeOnReadSnapshotRelation.getFilePath
import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath
import org.apache.hudi.common.model.{FileSlice, HoodieLogFile}
import org.apache.hudi.common.table.HoodieTableMetaClient
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.hadoop.utils.HoodieRealtimeRecordReaderUtils.getMaxCompactionMemoryInBytes
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.sources.Filter
@@ -63,14 +60,14 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
protected override def composeRDD(fileSplits: Seq[HoodieMergeOnReadFileSplit],
partitionSchema: StructType,
tableSchema: HoodieTableSchema,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Array[Filter]): HoodieMergeOnReadRDD = {
val fullSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
requiredSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = dataSchema,
// This file-reader is used to read base file records, subsequently merging them with the records
// stored in delta-log files. As such, we have to read _all_ records from the base file, while avoiding
// applying any filtering _before_ we complete combining them w/ delta-log records (to make sure that
@@ -85,7 +82,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
val requiredSchemaParquetReader = createBaseFileReader(
spark = sqlContext.sparkSession,
partitionSchema = partitionSchema,
tableSchema = tableSchema,
dataSchema = dataSchema,
requiredSchema = requiredSchema,
filters = filters,
options = optParams,
@@ -96,7 +93,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
val tableState = getTableState
new HoodieMergeOnReadRDD(sqlContext.sparkContext, jobConf, fullSchemaParquetReader, requiredSchemaParquetReader,
tableSchema, requiredSchema, tableState, mergeType, fileSplits)
dataSchema, requiredSchema, tableState, mergeType, fileSplits)
}
protected override def collectFileSplits(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): List[HoodieMergeOnReadFileSplit] = {

View File

@@ -120,6 +120,9 @@ class SparkHoodieTableFileIndex(spark: SparkSession,
StructType(schema.fields.filterNot(f => partitionColumns.contains(f.name)))
}
/**
* @VisibleForTesting
*/
def partitionSchema: StructType = {
if (queryAsNonePartitionedTable) {
// If we read it as Non-Partitioned table, we should not

View File

@@ -23,26 +23,32 @@ import org.apache.hudi.SparkAdapterSupport
import org.apache.spark.sql.SparkSession
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.execution.datasources.PartitionedFile
import org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat.FILE_FORMAT_ID
import org.apache.spark.sql.sources.Filter
import org.apache.spark.sql.types.StructType
class SparkHoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterSupport {
override def shortName(): String = "HoodieParquet"
class HoodieParquetFileFormat extends ParquetFileFormat with SparkAdapterSupport {
override def toString: String = "HoodieParquet"
override def shortName(): String = FILE_FORMAT_ID
override def buildReaderWithPartitionValues(
sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
override def toString: String = "Hoodie-Parquet"
override def buildReaderWithPartitionValues(sparkSession: SparkSession,
dataSchema: StructType,
partitionSchema: StructType,
requiredSchema: StructType,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
sparkAdapter
.createHoodieParquetFileFormat().get
.createHoodieParquetFileFormat(appendPartitionValues = false).get
.buildReaderWithPartitionValues(sparkSession, dataSchema, partitionSchema, requiredSchema, filters, options, hadoopConf)
}
}
object HoodieParquetFileFormat {
val FILE_FORMAT_ID = "hoodie-parquet"
}