1
0

[HUDI-3936] Fix projection for a nested field as pre-combined key (#5379)

This PR fixes the projection logic around a nested field which is used as the pre-combined key field. The fix is to only check and append the root level field for projection, i.e., "a", for a nested field "a.b.c" in the mandatory columns.

- Changes the logic to check and append the root level field for a required nested field in the mandatory columns in HoodieBaseRelation.appendMandatoryColumns
This commit is contained in:
Y Ethan Guo
2022-04-21 17:17:57 -07:00
committed by GitHub
parent 037f89ee7c
commit c4bc2deea0
8 changed files with 66 additions and 34 deletions

View File

@@ -26,7 +26,7 @@ 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.{HoodieParquetFileFormat, ParquetFileFormat}
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
@@ -54,8 +54,8 @@ 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
override lazy val mandatoryFields: Seq[String] =
// TODO reconcile, record's key shouldn't be mandatory for base-file only relation
Seq(recordKeyField)
override def imbueConfigs(sqlContext: SQLContext): Unit = {

View File

@@ -25,6 +25,7 @@ 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.HoodieConversionUtils.toScalaOption
import org.apache.hudi.avro.HoodieAvroUtils
import org.apache.hudi.common.config.{HoodieMetadataConfig, SerializableConfiguration}
import org.apache.hudi.common.fs.FSUtils
import org.apache.hudi.common.model.{HoodieFileFormat, HoodieRecord}
@@ -39,10 +40,8 @@ import org.apache.hudi.io.storage.HoodieHFileReader
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.codegen.GenerateUnsafeProjection
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression, UnsafeProjection}
import org.apache.spark.sql.catalyst.expressions.{Expression, SubqueryExpression}
import org.apache.spark.sql.execution.FileRelation
import org.apache.spark.sql.execution.datasources.{FileStatusCache, PartitionedFile, PartitioningUtils}
import org.apache.spark.sql.hudi.HoodieSqlCommonUtils
@@ -199,7 +198,10 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
*
* @VisibleInTests
*/
val mandatoryColumns: Seq[String]
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
@@ -246,7 +248,7 @@ 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] = appendMandatoryColumns(requiredColumns)
val fetchedColumns: Array[String] = appendMandatoryRootFields(requiredColumns)
val (requiredAvroSchema, requiredStructSchema, requiredInternalSchema) =
HoodieSparkUtils.getRequiredSchema(tableAvroSchema, fetchedColumns, internalSchema)
@@ -362,8 +364,11 @@ abstract class HoodieBaseRelation(val sqlContext: SQLContext,
!SubqueryExpression.hasSubquery(condition)
}
protected final def appendMandatoryColumns(requestedColumns: Array[String]): Array[String] = {
val missing = mandatoryColumns.filter(col => !requestedColumns.contains(col))
protected final def appendMandatoryRootFields(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))
requestedColumns ++ missing
}

View File

@@ -153,7 +153,7 @@ trait HoodieIncrementalRelationTrait extends HoodieBaseRelation {
Seq(isNotNullFilter, largerThanFilter, lessThanFilter)
}
override lazy val mandatoryColumns: Seq[String] = {
override lazy val mandatoryFields: Seq[String] = {
// NOTE: This columns are required for Incremental flow to be able to handle the rows properly, even in
// cases when no columns are requested to be fetched (for ex, when using {@code count()} API)
Seq(HoodieRecord.RECORD_KEY_METADATA_FIELD, HoodieRecord.COMMIT_TIME_METADATA_FIELD) ++

View File

@@ -47,7 +47,7 @@ class MergeOnReadSnapshotRelation(sqlContext: SQLContext,
override type FileSplit = HoodieMergeOnReadFileSplit
override lazy val mandatoryColumns: Seq[String] =
override lazy val mandatoryFields: Seq[String] =
Seq(recordKeyField) ++ preCombineFieldOpt.map(Seq(_)).getOrElse(Seq())
protected val mergeType: String = optParams.getOrElse(DataSourceReadOptions.REALTIME_MERGE.key,