1
0

[HUDI-3812] Fixing Data Skipping configuration to respect Metadata Table configs (#5244)

Addressing the problem of Data Skipping not respecting Metadata Table configs which might differ b/w write/read paths. More details could be found in HUDI-3812.

- Fixing Data Skipping configuration to respect MT configs (on the Read path)
- Tightening up DS handling of cases when no top-level columns are in the target query
- Enhancing tests to cover all possible case
This commit is contained in:
Alexey Kudinkin
2022-04-10 10:43:47 -07:00
committed by GitHub
parent 7a9d48d126
commit 976840e8eb
4 changed files with 119 additions and 75 deletions

View File

@@ -68,7 +68,7 @@ trait ColumnStatsIndexSupport extends SparkAdapterSupport {
if (targetColumns.nonEmpty) {
readColumnStatsIndexForColumnsInternal(spark, targetColumns, metadataConfig, tableBasePath)
} else {
readFullColumnStatsIndexInternal(spark, tableBasePath)
readFullColumnStatsIndexInternal(spark, metadataConfig, tableBasePath)
}
}
@@ -181,10 +181,11 @@ trait ColumnStatsIndexSupport extends SparkAdapterSupport {
spark.createDataFrame(transposedRDD, indexSchema)
}
private def readFullColumnStatsIndexInternal(spark: SparkSession, tableBasePath: String) = {
private def readFullColumnStatsIndexInternal(spark: SparkSession, metadataConfig: HoodieMetadataConfig, tableBasePath: String): DataFrame = {
val metadataTablePath = HoodieTableMetadata.getMetadataTableBasePath(tableBasePath)
// Read Metadata Table's Column Stats Index into Spark's [[DataFrame]]
spark.read.format("org.apache.hudi")
.options(metadataConfig.getProps.asScala)
.load(s"$metadataTablePath/${MetadataPartitionType.COLUMN_STATS.getPartitionPath}")
}

View File

@@ -85,11 +85,6 @@ case class HoodieFileIndex(spark: SparkSession,
override def rootPaths: Seq[Path] = queryPaths.asScala
def isDataSkippingEnabled: Boolean = {
options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(),
spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean
}
/**
* Returns the FileStatus for all the base files (excluding log files). This should be used only for
* cases where Spark directly fetches the list of files via HoodieFileIndex or for read optimized query logic
@@ -196,12 +191,20 @@ case class HoodieFileIndex(spark: SparkSession,
* @return list of pruned (data-skipped) candidate base-files' names
*/
private def lookupCandidateFilesInMetadataTable(queryFilters: Seq[Expression]): Try[Option[Set[String]]] = Try {
if (!isDataSkippingEnabled || queryFilters.isEmpty || !HoodieTableMetadataUtil.getCompletedMetadataPartitions(metaClient.getTableConfig)
.contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS)) {
// NOTE: Data Skipping is only effective when it references columns that are indexed w/in
// the Column Stats Index (CSI). Following cases could not be effectively handled by Data Skipping:
// - Expressions on top-level column's fields (ie, for ex filters like "struct.field > 0", since
// CSI only contains stats for top-level columns, in this case for "struct")
// - Any expression not directly referencing top-level column (for ex, sub-queries, since there's
// nothing CSI in particular could be applied for)
lazy val queryReferencedColumns = collectReferencedColumns(spark, queryFilters, schema)
if (!isMetadataTableEnabled || !isColumnStatsIndexAvailable || !isDataSkippingEnabled) {
validateConfig()
Option.empty
} else if (queryFilters.isEmpty || queryReferencedColumns.isEmpty) {
Option.empty
} else {
val queryReferencedColumns = collectReferencedColumns(spark, queryFilters, schema)
val colStatsDF: DataFrame = readColumnStatsIndex(spark, basePath, metadataConfig, queryReferencedColumns)
// Persist DF to avoid re-computing column statistics unraveling
@@ -245,13 +248,27 @@ case class HoodieFileIndex(spark: SparkSession,
override def refresh(): Unit = super.refresh()
override def inputFiles: Array[String] = {
val fileStatusList = allFiles
fileStatusList.map(_.getPath.toString).toArray
}
override def inputFiles: Array[String] =
allFiles.map(_.getPath.toString).toArray
override def sizeInBytes: Long = {
cachedFileSize
override def sizeInBytes: Long = cachedFileSize
private def isColumnStatsIndexAvailable =
HoodieTableMetadataUtil.getCompletedMetadataPartitions(metaClient.getTableConfig)
.contains(HoodieTableMetadataUtil.PARTITION_NAME_COLUMN_STATS)
private def isDataSkippingEnabled: Boolean =
options.getOrElse(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(),
spark.sessionState.conf.getConfString(DataSourceReadOptions.ENABLE_DATA_SKIPPING.key(), "false")).toBoolean
private def isMetadataTableEnabled: Boolean = metadataConfig.enabled()
private def isColumnStatsIndexEnabled: Boolean = metadataConfig.isColumnStatsIndexEnabled
private def validateConfig(): Unit = {
if (isDataSkippingEnabled && (!isMetadataTableEnabled || !isColumnStatsIndexEnabled)) {
logWarning("Data skipping requires both Metadata Table and Column Stats Index to be enabled as well! " +
s"(isMetadataTableEnabled = ${isMetadataTableEnabled}, isColumnStatsIndexEnabled = ${isColumnStatsIndexEnabled}")
}
}
}