1
0

[HUDI-3896] Porting Nested Schema Pruning optimization for Hudi's custom Relations (#5428)

Currently, all Hudi Relations bear performance gap relative to Spark's HadoopFsRelation 
and the reason to that is SchemaPruning optimization rule (pruning nested schemas) 
that is unfortunately predicated on usage of HadoopFsRelation, meaning that it's 
not applied in cases when any other relation is used.

This change is porting this rule to Hudi relations (MOR, Incremental, etc) 
by the virtue of leveraging HoodieSparkSessionExtensions mechanism 
injecting modified version of the original SchemaPruning rule 
that is adopted to work w/ Hudi's custom relations.

- Added customOptimizerRules to HoodieAnalysis
- Added NestedSchemaPrunning Spark's Optimizer rule
- Handle Spark's Optimizer pruned data schema (to effectively prune nested schemas)
- Enable HoodieClientTestHarness to inject HoodieSparkSessionExtensions
- Injecting Spark Session extensions for TestMORDataSource, TestCOWDataSource
- Disabled fallback to HadoopFsRelation
This commit is contained in:
Alexey Kudinkin
2022-07-21 02:36:06 -07:00
committed by GitHub
parent 2394c62973
commit de37774e12
42 changed files with 1220 additions and 500 deletions

View File

@@ -20,14 +20,12 @@ package org.apache.hudi
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.Path
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._
import org.apache.spark.sql.execution.datasources.parquet.HoodieParquetFileFormat
import org.apache.spark.sql.hive.orc.OrcFileFormat
import org.apache.spark.sql.sources.{BaseRelation, Filter}
import org.apache.spark.sql.types.StructType
@@ -59,10 +57,8 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
// For more details please check HUDI-4161
// NOTE: This override has to mirror semantic of whenever this Relation is converted into [[HadoopFsRelation]],
// which is currently done for all cases, except when Schema Evolution is enabled
override protected val shouldExtractPartitionValuesFromPartitionPath: Boolean = {
val enableSchemaOnRead = !internalSchema.isEmptySchema
!enableSchemaOnRead
}
override protected val shouldExtractPartitionValuesFromPartitionPath: Boolean =
internalSchemaOpt.isEmpty
override lazy val mandatoryFields: Seq[String] =
// TODO reconcile, record's key shouldn't be mandatory for base-file only relation
@@ -88,7 +84,7 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
hadoopConf = embedInternalSchema(new Configuration(conf), requiredSchema.internalSchema)
)
new HoodieFileScanRDD(sparkSession, baseFileReader, fileSplits)
@@ -124,16 +120,6 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
* rule; you can find more details in HUDI-3896)
*/
def toHadoopFsRelation: HadoopFsRelation = {
val (tableFileFormat, formatClassName) =
metaClient.getTableConfig.getBaseFileFormat match {
case HoodieFileFormat.ORC => (new OrcFileFormat, "orc")
case HoodieFileFormat.PARQUET =>
// 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 parquetFileFormat = sparkAdapter.createHoodieParquetFileFormat(shouldExtractPartitionValuesFromPartitionPath).get
(parquetFileFormat, HoodieParquetFileFormat.FILE_FORMAT_ID)
}
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
@@ -157,27 +143,46 @@ class BaseFileOnlyRelation(sqlContext: SQLContext,
partitionSchema = partitionSchema,
dataSchema = dataSchema,
bucketSpec = None,
fileFormat = tableFileFormat,
fileFormat = fileFormat,
optParams)(sparkSession)
} else {
val readPathsStr = optParams.get(DataSourceReadOptions.READ_PATHS.key)
val extraReadPaths = readPathsStr.map(p => p.split(",").toSeq).getOrElse(Seq())
// NOTE: Spark is able to infer partitioning values from partition path only when Hive-style partitioning
// scheme is used. Therefore, we fallback to reading the table as non-partitioned (specifying
// partitionColumns = Seq.empty) whenever Hive-style partitioning is not involved
val partitionColumns: Seq[String] = if (tableConfig.getHiveStylePartitioningEnable.toBoolean) {
this.partitionColumns
} else {
Seq.empty
}
DataSource.apply(
sparkSession = sparkSession,
paths = extraReadPaths,
// Here we should specify the schema to the latest commit schema since
// the table schema evolution.
userSpecifiedSchema = userSchema.orElse(Some(tableStructSchema)),
className = formatClassName,
// Since we're reading the table as just collection of files we have to make sure
// we only read the latest version of every Hudi's file-group, which might be compacted, clustered, etc.
// while keeping previous versions of the files around as well.
//
// We rely on [[HoodieROTablePathFilter]], to do proper filtering to assure that
className = fileFormatClassName,
options = optParams ++ Map(
"mapreduce.input.pathFilter.class" -> classOf[HoodieROTablePathFilter].getName
)
// Since we're reading the table as just collection of files we have to make sure
// we only read the latest version of every Hudi's file-group, which might be compacted, clustered, etc.
// while keeping previous versions of the files around as well.
//
// We rely on [[HoodieROTablePathFilter]], to do proper filtering to assure that
"mapreduce.input.pathFilter.class" -> classOf[HoodieROTablePathFilter].getName,
// We have to override [[EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH]] setting, since
// the relation might have this setting overridden
DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key -> shouldExtractPartitionValuesFromPartitionPath.toString,
// NOTE: We have to specify table's base-path explicitly, since we're requesting Spark to read it as a
// list of globbed paths which complicates partitioning discovery for Spark.
// Please check [[PartitioningAwareFileIndex#basePaths]] comment for more details.
PartitioningAwareFileIndex.BASE_PATH_PARAM -> metaClient.getBasePathV2.toString
),
partitionColumns = partitionColumns
)
.resolveRelation()
.asInstanceOf[HadoopFsRelation]

View File

@@ -26,6 +26,7 @@ import org.apache.hudi.common.model.HoodieTableType.{COPY_ON_WRITE, MERGE_ON_REA
import org.apache.hudi.common.table.timeline.HoodieInstant
import org.apache.hudi.common.table.{HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.exception.HoodieException
import org.apache.hudi.metadata.HoodieTableMetadata.isMetadataTable
import org.apache.log4j.LogManager
import org.apache.spark.sql.execution.streaming.{Sink, Source}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils.isUsingHiveCatalog
@@ -127,6 +128,7 @@ class DefaultSource extends RelationProvider
(COPY_ON_WRITE, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) |
(MERGE_ON_READ, QUERY_TYPE_READ_OPTIMIZED_OPT_VAL, false) =>
resolveBaseFileOnlyRelation(sqlContext, globPaths, userSchema, metaClient, parameters)
case (COPY_ON_WRITE, QUERY_TYPE_INCREMENTAL_OPT_VAL, _) =>
new IncrementalRelation(sqlContext, parameters, userSchema, metaClient)

View File

@@ -23,10 +23,11 @@ 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.{convertToAvroSchema, createHFileReader, generateUnsafeProjection, getPartitionPath}
import org.apache.hudi.HoodieBaseRelation.{convertToAvroSchema, createHFileReader, generateUnsafeProjection, getPartitionPath, projectSchema}
import org.apache.hudi.HoodieConversionUtils.toScalaOption
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.config.HoodieMetadataConfig
import org.apache.hudi.client.utils.SparkInternalSchemaConverter
import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration}
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.fs.FSUtils.getRelativePartitionPath
import org.apache.hudi.common.model.{FileSlice, HoodieFileFormat, HoodieRecord}
@@ -34,18 +35,21 @@ import org.apache.hudi.common.table.timeline.{HoodieInstant, HoodieTimeline}
import org.apache.hudi.common.table.view.HoodieTableFileSystemView
import org.apache.hudi.common.table.{HoodieTableConfig, HoodieTableMetaClient, TableSchemaResolver}
import org.apache.hudi.common.util.StringUtils
import org.apache.hudi.common.util.StringUtils.isNullOrEmpty
import org.apache.hudi.common.util.ValidationUtils.checkState
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
import org.apache.hudi.internal.schema.{HoodieSchemaException, InternalSchema}
import org.apache.hudi.internal.schema.convert.AvroInternalSchemaConverter
import org.apache.hudi.internal.schema.utils.{InternalSchemaUtils, SerDeHelper}
import org.apache.hudi.io.storage.HoodieHFileReader
import org.apache.spark.SerializableWritable
import org.apache.spark.execution.datasources.HoodieInMemoryFileIndex
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
import org.apache.spark.sql.execution.FileRelation
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionDirectory, PartitionedFile, PartitioningUtils}
import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat
import org.apache.spark.sql.execution.datasources.parquet.{HoodieParquetFileFormat, ParquetFileFormat}
import org.apache.spark.sql.execution.datasources.{FileFormat, FileStatusCache, PartitionDirectory, PartitionedFile, PartitioningUtils}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
import org.apache.spark.sql.sources.{BaseRelation, Filter, PrunedFilteredScan}
import org.apache.spark.sql.types.{StringType, StructField, StructType}
@@ -60,7 +64,7 @@ import scala.util.{Failure, Success, Try}
trait HoodieFileSplit {}
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: InternalSchema = InternalSchema.getEmptyInternalSchema)
case class HoodieTableSchema(structTypeSchema: StructType, avroSchemaStr: String, internalSchema: Option[InternalSchema] = None)
case class HoodieTableState(tablePath: String,
latestCommitTimestamp: String,
@@ -80,7 +84,8 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
extends BaseRelation
with FileRelation
with PrunedFilteredScan
with Logging {
with Logging
with SparkAdapterSupport {
type FileSplit <: HoodieFileSplit
@@ -128,9 +133,24 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
* NOTE: Initialization of teh following members is coupled on purpose to minimize amount of I/O
* required to fetch table's Avro and Internal schemas
*/
protected lazy val (tableAvroSchema: Schema, internalSchema: InternalSchema) = {
protected lazy val (tableAvroSchema: Schema, internalSchemaOpt: Option[InternalSchema]) = {
val schemaResolver = new TableSchemaResolver(metaClient)
val avroSchema: Schema = schemaSpec.map(convertToAvroSchema).getOrElse {
val internalSchemaOpt = if (!isSchemaEvolutionEnabled) {
None
} else {
Try(schemaResolver.getTableInternalSchemaFromCommitMetadata) match {
case Success(internalSchemaOpt) => toScalaOption(internalSchemaOpt)
case Failure(e) =>
logWarning("Failed to fetch internal-schema from the table", e)
None
}
}
val avroSchema = internalSchemaOpt.map { is =>
AvroInternalSchemaConverter.convert(is, "schema")
} orElse {
schemaSpec.map(convertToAvroSchema)
} getOrElse {
Try(schemaResolver.getTableAvroSchema) match {
case Success(schema) => schema
case Failure(e) =>
@@ -139,25 +159,20 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
}
}
val internalSchema: InternalSchema = if (!isSchemaEvolutionEnabled) {
InternalSchema.getEmptyInternalSchema
} else {
Try(schemaResolver.getTableInternalSchemaFromCommitMetadata) match {
case Success(internalSchemaOpt) =>
toScalaOption(internalSchemaOpt).getOrElse(InternalSchema.getEmptyInternalSchema)
case Failure(e) =>
logWarning("Failed to fetch internal-schema from the table", e)
InternalSchema.getEmptyInternalSchema
}
}
(avroSchema, internalSchema)
(avroSchema, internalSchemaOpt)
}
protected lazy val tableStructSchema: StructType = AvroConversionUtils.convertAvroSchemaToStructType(tableAvroSchema)
protected val partitionColumns: Array[String] = tableConfig.getPartitionFields.orElse(Array.empty)
/**
* Data schema optimized (externally) by Spark's Optimizer.
*
* Please check scala-doc for [[updatePrunedDataSchema]] more details
*/
protected var optimizerPrunedDataSchema: Option[StructType] = None
/**
* Controls whether partition values (ie values of partition columns) should be
* <ol>
@@ -189,6 +204,16 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
shouldOmitPartitionColumns || shouldExtractPartitionValueFromPath
}
lazy val (fileFormat: FileFormat, fileFormatClassName: String) =
metaClient.getTableConfig.getBaseFileFormat match {
case HoodieFileFormat.ORC => (new OrcFileFormat, "orc")
case HoodieFileFormat.PARQUET =>
// 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 parquetFileFormat = sparkAdapter.createHoodieParquetFileFormat(shouldExtractPartitionValuesFromPartitionPath).get
(parquetFileFormat, HoodieParquetFileFormat.FILE_FORMAT_ID)
}
/**
* NOTE: PLEASE READ THIS CAREFULLY
*
@@ -206,19 +231,14 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
* meaning that regardless of whether this columns are being requested by the query they will be fetched
* regardless so that relation is able to combine records properly (if necessary)
*
* @VisibleForTesting
* @VisibleInTests
*/
val mandatoryFields: Seq[String]
protected def mandatoryRootFields: Seq[String] =
mandatoryFields.map(col => HoodieAvroUtils.getRootLevelFieldName(col))
protected def timeline: HoodieTimeline =
// NOTE: We're including compaction here since it's not considering a "commit" operation
metaClient.getCommitsAndCompactionTimeline.filterCompletedInstants
protected val validCommits = timeline.getInstants.toArray().map(_.asInstanceOf[HoodieInstant].getFileName).mkString(",")
protected def latestInstant: Option[HoodieInstant] =
toScalaOption(timeline.lastInstant())
@@ -228,9 +248,38 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
/**
* Returns true in case table supports Schema on Read (Schema Evolution)
*/
def hasSchemaOnRead: Boolean = !internalSchema.isEmptySchema
def hasSchemaOnRead: Boolean = internalSchemaOpt.isDefined
override def schema: StructType = tableStructSchema
/**
* Data schema is determined as the actual schema of the Table's Data Files (for ex, parquet/orc/etc);
*
* In cases when partition values are not persisted w/in the data files, data-schema is defined as
* <pre>table's schema - partition columns</pre>
*
* Check scala-doc for [[shouldExtractPartitionValuesFromPartitionPath]] for more details
*/
def dataSchema: StructType =
if (shouldExtractPartitionValuesFromPartitionPath) {
prunePartitionColumns(tableStructSchema)
} else {
tableStructSchema
}
/**
* Determines whether relation's schema could be pruned by Spark's Optimizer
*/
def canPruneRelationSchema: Boolean =
(fileFormat.isInstanceOf[ParquetFileFormat] || fileFormat.isInstanceOf[OrcFileFormat]) &&
// NOTE: Some relations might be disabling sophisticated schema pruning techniques (for ex, nested schema pruning)
// TODO(HUDI-XXX) internal schema doesn't supported nested schema pruning currently
!hasSchemaOnRead
override def schema: StructType = {
// NOTE: Optimizer could prune the schema (applying for ex, [[NestedSchemaPruning]] rule) setting new updated
// schema in-place (via [[setPrunedDataSchema]] method), therefore we have to make sure that we pick
// pruned data schema (if present) over the standard table's one
optimizerPrunedDataSchema.getOrElse(tableStructSchema)
}
/**
* This method controls whether relation will be producing
@@ -263,22 +312,24 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
//
// (!!!) IT'S CRITICAL TO AVOID REORDERING OF THE REQUESTED COLUMNS AS THIS WILL BREAK THE UPSTREAM
// PROJECTION
val fetchedColumns: Array[String] = appendMandatoryRootFields(requiredColumns)
val targetColumns: Array[String] = appendMandatoryColumns(requiredColumns)
// NOTE: We explicitly fallback to default table's Avro schema to make sure we avoid unnecessary Catalyst > Avro
// schema conversion, which is lossy in nature (for ex, it doesn't preserve original Avro type-names) and
// could have an effect on subsequent de-/serializing records in some exotic scenarios (when Avro unions
// w/ more than 2 types are involved)
val sourceSchema = optimizerPrunedDataSchema.map(convertToAvroSchema).getOrElse(tableAvroSchema)
val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) =
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns, internalSchema)
projectSchema(Either.cond(internalSchemaOpt.isDefined, internalSchemaOpt.get, sourceSchema), targetColumns)
val filterExpressions = convertToExpressions(filters)
val (partitionFilters, dataFilters) = filterExpressions.partition(isPartitionPredicate)
val fileSplits = collectFileSplits(partitionFilters, dataFilters)
val tableAvroSchemaStr =
if (internalSchema.isEmptySchema) tableAvroSchema.toString
else AvroInternalSchemaConverter.convert(internalSchema, tableAvroSchema.getName).toString
val tableAvroSchemaStr = tableAvroSchema.toString
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchemaStr, internalSchema)
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, requiredInternalSchema)
val tableSchema = HoodieTableSchema(tableStructSchema, tableAvroSchemaStr, internalSchemaOpt)
val requiredSchema = HoodieTableSchema(requiredStructSchema, requiredAvroSchema.toString, Some(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
@@ -289,19 +340,19 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
// 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) =
val (partitionSchema, dataSchema, requiredDataSchema) =
tryPrunePartitionColumns(tableSchema, requiredSchema)
if (fileSplits.isEmpty) {
sparkSession.sparkContext.emptyRDD
} else {
val rdd = composeRDD(fileSplits, partitionSchema, dataSchema, prunedRequiredSchema, filters)
val rdd = composeRDD(fileSplits, partitionSchema, dataSchema, requiredDataSchema, 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) {
val projectedRDD = if (requiredDataSchema.structTypeSchema != requiredSchema.structTypeSchema) {
rdd.mapPartitions { it =>
val fullPrunedSchema = StructType(prunedRequiredSchema.structTypeSchema.fields ++ partitionSchema.fields)
val fullPrunedSchema = StructType(requiredDataSchema.structTypeSchema.fields ++ partitionSchema.fields)
val unsafeProjection = generateUnsafeProjection(fullPrunedSchema, requiredSchema.structTypeSchema)
it.map(unsafeProjection)
}
@@ -408,11 +459,12 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
!SubqueryExpression.hasSubquery(condition)
}
protected final def appendMandatoryRootFields(requestedColumns: Array[String]): Array[String] = {
protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
// For a nested field in mandatory columns, we should first get the root-level field, and then
// check for any missing column, as the requestedColumns should only contain root-level fields
// We should only append root-level field as well
val missing = mandatoryRootFields.filter(rootField => !requestedColumns.contains(rootField))
val missing = mandatoryFields.map(col => HoodieAvroUtils.getRootLevelFieldName(col))
.filter(rootField => !requestedColumns.contains(rootField))
requestedColumns ++ missing
}
@@ -476,6 +528,19 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
}
}
/**
* Hook for Spark's Optimizer to update expected relation schema after pruning
*
* NOTE: Only limited number of optimizations in respect to schema pruning could be performed
* internally w/in the relation itself w/o consideration for how the relation output is used.
* Therefore more advanced optimizations (like [[NestedSchemaPruning]]) have to be carried out
* by Spark's Optimizer holistically evaluating Spark's [[LogicalPlan]]
*/
def updatePrunedDataSchema(prunedSchema: StructType): this.type = {
optimizerPrunedDataSchema = Some(prunedSchema)
this
}
/**
* Returns file-reader routine accepting [[PartitionedFile]] and returning an [[Iterator]]
* over [[InternalRow]]
@@ -521,6 +586,19 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
}
}
protected def embedInternalSchema(conf: Configuration, internalSchemaOpt: Option[InternalSchema]): Configuration = {
val internalSchema = internalSchemaOpt.getOrElse(InternalSchema.getEmptyInternalSchema)
val querySchemaString = SerDeHelper.toJson(internalSchema)
if (!isNullOrEmpty(querySchemaString)) {
val validCommits = timeline.getInstants.iterator.asScala.map(_.getFileName).mkString(",")
conf.set(SparkInternalSchemaConverter.HOODIE_QUERY_SCHEMA, SerDeHelper.toJson(internalSchema))
conf.set(SparkInternalSchemaConverter.HOODIE_TABLE_PATH, metaClient.getBasePath)
conf.set(SparkInternalSchemaConverter.HOODIE_VALID_COMMITS_LIST, validCommits)
}
conf
}
private def tryPrunePartitionColumns(tableSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema): (StructType, HoodieTableSchema, HoodieTableSchema) = {
if (shouldExtractPartitionValuesFromPartitionPath) {
@@ -544,15 +622,15 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
// t/h Spark Session configuration (for ex, for Spark SQL)
optParams.getOrElse(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean ||
sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
sparkSession.conf.get(DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.key,
DataSourceReadOptions.SCHEMA_EVOLUTION_ENABLED.defaultValue.toString).toBoolean
}
}
object HoodieBaseRelation extends SparkAdapterSupport {
private def generateUnsafeProjection(from: StructType, to: StructType) =
sparkAdapter.createCatalystExpressionUtils().generateUnsafeProjection(from, to)
sparkAdapter.getCatalystExpressionUtils().generateUnsafeProjection(from, to)
def convertToAvroSchema(structSchema: StructType): Schema =
sparkAdapter.getAvroSchemaConverters.toAvroType(structSchema, nullable = false, "Record")
@@ -560,16 +638,50 @@ object HoodieBaseRelation extends SparkAdapterSupport {
def getPartitionPath(fileStatus: FileStatus): Path =
fileStatus.getPath.getParent
/**
* Projects provided schema by picking only required (projected) top-level columns from it
*
* @param tableSchema schema to project (either of [[InternalSchema]] or Avro's [[Schema]])
* @param requiredColumns required top-level columns to be projected
*/
def projectSchema(tableSchema: Either[Schema, InternalSchema], requiredColumns: Array[String]): (Schema, StructType, InternalSchema) = {
tableSchema match {
case Right(internalSchema) =>
checkState(!internalSchema.isEmptySchema)
// TODO extend pruning to leverage optimizer pruned schema
val prunedInternalSchema = InternalSchemaUtils.pruneInternalSchema(internalSchema, requiredColumns.toList.asJava)
val requiredAvroSchema = AvroInternalSchemaConverter.convert(prunedInternalSchema, "schema")
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
(requiredAvroSchema, requiredStructSchema, prunedInternalSchema)
case Left(avroSchema) =>
val fieldMap = avroSchema.getFields.asScala.map(f => f.name() -> f).toMap
val requiredFields = requiredColumns.map { col =>
val f = fieldMap(col)
// We have to create a new [[Schema.Field]] since Avro schemas can't share field
// instances (and will throw "org.apache.avro.AvroRuntimeException: Field already used")
new Schema.Field(f.name(), f.schema(), f.doc(), f.defaultVal(), f.order())
}.toList
val requiredAvroSchema = Schema.createRecord(avroSchema.getName, avroSchema.getDoc,
avroSchema.getNamespace, avroSchema.isError, requiredFields.asJava)
val requiredStructSchema = AvroConversionUtils.convertAvroSchemaToStructType(requiredAvroSchema)
(requiredAvroSchema, requiredStructSchema, InternalSchema.getEmptyInternalSchema)
}
}
private def createHFileReader(spark: SparkSession,
dataSchema: HoodieTableSchema,
requiredSchema: HoodieTableSchema,
filters: Seq[Filter],
options: Map[String, String],
hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = {
val hadoopConfBroadcast = spark.sparkContext.broadcast(new SerializableWritable(hadoopConf))
val hadoopConfBroadcast =
spark.sparkContext.broadcast(new SerializableConfiguration(hadoopConf))
partitionedFile => {
val hadoopConf = hadoopConfBroadcast.value.value
val hadoopConf = hadoopConfBroadcast.value.get()
val reader = new HoodieHFileReader[GenericRecord](hadoopConf, new Path(partitionedFile.filePath),
new CacheConfig(hadoopConf))
@@ -581,7 +693,7 @@ object HoodieBaseRelation extends SparkAdapterSupport {
reader.getRecordIterator(requiredAvroSchema).asScala
.map(record => {
avroToRowConverter.apply(record.asInstanceOf[GenericRecord]).get
avroToRowConverter.apply(record).get
})
}
}

View File

@@ -82,23 +82,4 @@ object HoodieDataSourceHelper extends PredicateHelper with SparkAdapterSupport {
PartitionedFile(partitionValues, filePath.toUri.toString, offset, size)
}
}
/**
* Set internalSchema evolution parameters to configuration.
* spark will broadcast them to each executor, we use those parameters to do schema evolution.
*
* @param conf hadoop conf.
* @param internalSchema internalschema for query.
* @param tablePath hoodie table base path.
* @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 = {
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

@@ -166,10 +166,11 @@ class HoodieMergeOnReadRDD(@transient sc: SparkContext,
// be stored in non-columnar formats like Avro, HFile, etc)
private val requiredSchemaFieldOrdinals: List[Int] = collectFieldOrdinals(requiredAvroSchema, logFileReaderAvroSchema)
// TODO: now logScanner with internalSchema support column project, we may no need projectAvroUnsafe
private var logScanner =
private var logScanner = {
val internalSchema = tableSchema.internalSchema.getOrElse(InternalSchema.getEmptyInternalSchema)
HoodieMergeOnReadRDD.scanLog(split.logFiles, getPartitionPath(split), logFileReaderAvroSchema, tableState,
maxCompactionMemoryInBytes, config, tableSchema.internalSchema)
maxCompactionMemoryInBytes, config, internalSchema)
}
private val logRecords = logScanner.getRecords.asScala

View File

@@ -78,7 +78,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
hadoopConf = embedInternalSchema(new Configuration(conf), internalSchemaOpt)
)
val requiredSchemaParquetReader = createBaseFileReader(
@@ -90,7 +90,7 @@ class MergeOnReadIncrementalRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
hadoopConf = embedInternalSchema(new Configuration(conf), requiredSchema.internalSchema)
)
val hoodieTableState = getTableState

View File

@@ -76,7 +76,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), internalSchema, metaClient.getBasePath, validCommits)
hadoopConf = embedInternalSchema(new Configuration(conf), internalSchemaOpt)
)
val requiredSchemaParquetReader = createBaseFileReader(
@@ -88,7 +88,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
options = optParams,
// NOTE: We have to fork the Hadoop Config here as Spark will be modifying it
// to configure Parquet reader appropriately
hadoopConf = HoodieDataSourceHelper.getConfigurationWithInternalSchema(new Configuration(conf), requiredSchema.internalSchema, metaClient.getBasePath, validCommits)
hadoopConf = embedInternalSchema(new Configuration(conf), requiredSchema.internalSchema)
)
val tableState = getTableState

View File

@@ -56,6 +56,8 @@ private[sql] object SchemaConverters {
toSqlTypeHelper(avroSchema, Set.empty)
}
private val unionFieldMemberPrefix = "member"
private def toSqlTypeHelper(avroSchema: Schema, existingRecordNames: Set[String]): SchemaType = {
avroSchema.getType match {
case INT => avroSchema.getLogicalType match {
@@ -134,7 +136,7 @@ private[sql] object SchemaConverters {
case (s, i) =>
val schemaType = toSqlTypeHelper(s, existingRecordNames)
// All fields are nullable because only one of them is set at a time
StructField(s"member$i", schemaType.dataType, nullable = true)
StructField(s"$unionFieldMemberPrefix$i", schemaType.dataType, nullable = true)
}
SchemaType(StructType(fields.toSeq), nullable = false)
@@ -187,23 +189,46 @@ private[sql] object SchemaConverters {
.values(toAvroType(vt, valueContainsNull, recordName, nameSpace))
case st: StructType =>
val childNameSpace = if (nameSpace != "") s"$nameSpace.$recordName" else recordName
val fieldsAssembler = builder.record(recordName).namespace(nameSpace).fields()
st.foreach { f =>
val fieldAvroType =
toAvroType(f.dataType, f.nullable, f.name, childNameSpace)
fieldsAssembler.name(f.name).`type`(fieldAvroType).noDefault()
if (canBeUnion(st)) {
val nonNullUnionFieldTypes = st.map(f => toAvroType(f.dataType, nullable = false, f.name, childNameSpace))
val unionFieldTypes = if (nullable) {
nullSchema +: nonNullUnionFieldTypes
} else {
nonNullUnionFieldTypes
}
Schema.createUnion(unionFieldTypes:_*)
} else {
val fieldsAssembler = builder.record(recordName).namespace(nameSpace).fields()
st.foreach { f =>
val fieldAvroType =
toAvroType(f.dataType, f.nullable, f.name, childNameSpace)
fieldsAssembler.name(f.name).`type`(fieldAvroType).noDefault()
}
fieldsAssembler.endRecord()
}
fieldsAssembler.endRecord()
// This should never happen.
case other => throw new IncompatibleSchemaException(s"Unexpected type $other.")
}
if (nullable && catalystType != NullType) {
if (nullable && catalystType != NullType && schema.getType != Schema.Type.UNION) {
Schema.createUnion(schema, nullSchema)
} else {
schema
}
}
private def canBeUnion(st: StructType): Boolean = {
// We use a heuristic to determine whether a [[StructType]] could potentially have been produced
// by converting Avro union to Catalyst's [[StructType]]:
// - It has to have at least 1 field
// - All fields have to be of the following format "memberN" (where N is sequentially increasing integer)
// - All fields are nullable
st.fields.length > 0 &&
st.forall { f =>
f.name.matches(s"$unionFieldMemberPrefix\\d+") && f.nullable
}
}
}
private[avro] class IncompatibleSchemaException(msg: String, ex: Throwable = null) extends Exception(msg, ex)

View File

@@ -384,7 +384,7 @@ private object ColumnStatsExpressionUtils {
* Returns only [[AttributeReference]] contained as a sub-expression
*/
object AllowedTransformationExpression extends SparkAdapterSupport {
val exprUtils: HoodieCatalystExpressionUtils = sparkAdapter.createCatalystExpressionUtils()
val exprUtils: HoodieCatalystExpressionUtils = sparkAdapter.getCatalystExpressionUtils()
def unapply(expr: Expression): Option[AttributeReference] = {
// First step, we check that expression

View File

@@ -53,7 +53,7 @@ object HoodieSqlCommonUtils extends SparkAdapterSupport {
def getTableIdentifier(table: LogicalPlan): TableIdentifier = {
table match {
case SubqueryAlias(name, _) => sparkAdapter.toTableIdentifier(name)
case SubqueryAlias(name, _) => sparkAdapter.getCatalystPlanUtils.toTableIdentifier(name)
case _ => throw new IllegalArgumentException(s"Illegal table: $table")
}
}