From 5b944159202c54e009cfc09bf1d6bdfaf325813a Mon Sep 17 00:00:00 2001 From: Vinish Reddy Date: Tue, 30 Dec 2025 17:31:26 -0800 Subject: [PATCH 01/20] Handle nested map and array columns in MDT --- .../hudi/SparkHoodieTableFileIndex.scala | 4 +- ...HoodieFileGroupReaderBasedFileFormat.scala | 1 + .../hudi/functional/TestCOWDataSource.scala | 65 +++++++++++++++++++ 3 files changed, 69 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 1ba9628af3b75..808493240f786 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -36,7 +36,7 @@ import org.apache.hudi.hadoop.HoodieLatestBaseFilesPathFilter import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.internal.schema.Types.RecordType import org.apache.hudi.internal.schema.utils.Conversions -import org.apache.hudi.keygen.{StringPartitionPathFormatter, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} +import org.apache.hudi.keygen.{CustomAvroKeyGenerator, CustomKeyGenerator, StringPartitionPathFormatter, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.metadata.{CatalogBackedTableMetadata, HoodieTableMetadata} import org.apache.hudi.storage.{StoragePath, StoragePathInfo} import org.apache.hudi.util.JFunction @@ -147,6 +147,8 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val keyGeneratorClassName = tableConfig.getKeyGeneratorClassName if (classOf[TimestampBasedKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName) || classOf[TimestampBasedAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) { + // || classOf[CustomKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName) + // || classOf[CustomAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) { val partitionFields: Array[StructField] = partitionColumns.get().map(column => StructField(column, StringType)) StructType(partitionFields) } else { diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index de4ffb400d4c2..e81ae9adc63f9 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -265,6 +265,7 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val exclusionFields = new java.util.HashSet[String]() exclusionFields.add("op") partitionSchema.fields.foreach(f => exclusionFields.add(f.name)) + // QQ: How do we add nested partitioned fields to this without any field id's? val requestedStructType = StructType(requiredSchema.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name))) val requestedSchema = HoodieSchemaUtils.pruneDataSchema(schema, HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(requestedStructType, sanitizedTableName), exclusionFields) val dataStructTypeWithMandatoryPartitionFields = StructType(dataStructType.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name))) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index db5ca1ee78002..4b82563b85950 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -2616,6 +2616,71 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals("row3", results(2).getAs[String]("_row_key")) assertEquals("value3", results(2).getAs[String]("data")) } + + @Test + def testNestedFieldPartition(): Unit = { + // Define schema with nested_record containing level field + val nestedSchema = StructType(Seq( + StructField("nested_int", IntegerType, nullable = false), + StructField("level", StringType, nullable = false) + )) + + val schema = StructType(Seq( + StructField("key", StringType, nullable = false), + StructField("ts", LongType, nullable = false), + StructField("level", StringType, nullable = false), + StructField("int_field", IntegerType, nullable = false), + StructField("string_field", StringType, nullable = true), + StructField("nested_record", nestedSchema, nullable = true) + )) + + // Create test data where top-level 'level' and 'nested_record.level' have DIFFERENT values + // This helps verify we're correctly partitioning/filtering on the nested field + val records = Seq( + Row("key1", 1L, "L1", 1, "str1", Row(10, "INFO")), + Row("key2", 2L, "L2", 2, "str2", Row(20, "ERROR")), + Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), + Row("key4", 4L, "L4", 4, "str4", Row(40, "DEBUG")), + Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")) + ) + + val inputDF = spark.createDataFrame( + spark.sparkContext.parallelize(records), + schema + ) + + // Write to Hudi partitioned by nested_record.level + inputDF.write.format("hudi") + .option("hoodie.insert.shuffle.parallelism", "4") + .option("hoodie.upsert.shuffle.parallelism", "4") + .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "key") + .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "nested_record.level") + .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts") + .option(HoodieWriteConfig.TBL_NAME.key, "test_nested_partition") + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Overwrite) + .save(basePath) + + // Read and filter on nested_record.level = 'INFO' + val results = spark.read.format("hudi") + .load(basePath) + .filter("nested_record.level = 'INFO'") + .select("key", "ts", "level", "int_field", "string_field", "nested_record") + .orderBy("key") + .collect() + + // Expected results - 3 records with nested_record.level = 'INFO' + val expectedResults = Array( + Row("key1", 1L, "L1", 1, "str1", Row(10, "INFO")), + Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), + Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")) + ) + + assertEquals(expectedResults.length, results.length) + expectedResults.zip(results).foreach { case (expected, actual) => + assertEquals(expected, actual) + } + } } object TestCOWDataSource { From 0e26dc853cf659aac7711f8c241162f621ca378b Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Sat, 7 Feb 2026 16:56:23 -0800 Subject: [PATCH 02/20] Fix the issue and add tests --- .../hudi/SparkHoodieTableFileIndex.scala | 7 +- ...HoodieFileGroupReaderBasedFileFormat.scala | 1 - .../hudi/functional/TestCOWDataSource.scala | 173 ++++++++++++++++-- 3 files changed, 157 insertions(+), 24 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 808493240f786..e4b79bcfe0842 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -147,13 +147,14 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val keyGeneratorClassName = tableConfig.getKeyGeneratorClassName if (classOf[TimestampBasedKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName) || classOf[TimestampBasedAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) { - // || classOf[CustomKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName) - // || classOf[CustomAvroKeyGenerator].getName.equalsIgnoreCase(keyGeneratorClassName)) { val partitionFields: Array[StructField] = partitionColumns.get().map(column => StructField(column, StringType)) StructType(partitionFields) } else { + // Use full partition path (e.g. "nested_record.level") as the partition column name so that + // data schema does not exclude a same-named top-level column (e.g. "level") when partition + // path is a nested field. Otherwise partition value would overwrite the data column on read. val partitionFields: Array[StructField] = partitionColumns.get().filter(column => nameFieldMap.contains(column)) - .map(column => nameFieldMap.apply(column)) + .map(column => StructField(column, nameFieldMap.apply(column).dataType)) if (partitionFields.length != partitionColumns.get().length) { val isBootstrapTable = tableConfig.getBootstrapBasePath.isPresent diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala index e81ae9adc63f9..de4ffb400d4c2 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/HoodieFileGroupReaderBasedFileFormat.scala @@ -265,7 +265,6 @@ class HoodieFileGroupReaderBasedFileFormat(tablePath: String, val exclusionFields = new java.util.HashSet[String]() exclusionFields.add("op") partitionSchema.fields.foreach(f => exclusionFields.add(f.name)) - // QQ: How do we add nested partitioned fields to this without any field id's? val requestedStructType = StructType(requiredSchema.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name))) val requestedSchema = HoodieSchemaUtils.pruneDataSchema(schema, HoodieSchemaConversionUtils.convertStructTypeToHoodieSchema(requestedStructType, sanitizedTableName), exclusionFields) val dataStructTypeWithMandatoryPartitionFields = StructType(dataStructType.fields ++ partitionSchema.fields.filter(f => mandatoryFields.contains(f.name))) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 4b82563b85950..3eb44154ebf00 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -2617,8 +2617,9 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals("value3", results(2).getAs[String]("data")) } - @Test - def testNestedFieldPartition(): Unit = { + @ParameterizedTest + @CsvSource(Array("COW", "MOR")) + def testNestedFieldPartition(tableType: String): Unit = { // Define schema with nested_record containing level field val nestedSchema = StructType(Seq( StructField("nested_int", IntegerType, nullable = false), @@ -2636,7 +2637,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup // Create test data where top-level 'level' and 'nested_record.level' have DIFFERENT values // This helps verify we're correctly partitioning/filtering on the nested field - val records = Seq( + val recordsCommit1 = Seq( Row("key1", 1L, "L1", 1, "str1", Row(10, "INFO")), Row("key2", 2L, "L2", 2, "str2", Row(20, "ERROR")), Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), @@ -2644,40 +2645,172 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")) ) - val inputDF = spark.createDataFrame( - spark.sparkContext.parallelize(records), - schema + val tableTypeOptVal = if (tableType == "MOR") { + DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL + } else { + DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL + } + + val baseWriteOpts = Map( + "hoodie.insert.shuffle.parallelism" -> "4", + "hoodie.upsert.shuffle.parallelism" -> "4", + DataSourceWriteOptions.RECORDKEY_FIELD.key -> "key", + DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "nested_record.level", + HoodieTableConfig.ORDERING_FIELDS.key -> "ts", + HoodieWriteConfig.TBL_NAME.key -> "test_nested_partition", + DataSourceWriteOptions.TABLE_TYPE.key -> tableTypeOptVal ) + val writeOpts = if (tableType == "MOR") { + baseWriteOpts + ("hoodie.compact.inline" -> "false") + } else { + baseWriteOpts + } - // Write to Hudi partitioned by nested_record.level - inputDF.write.format("hudi") - .option("hoodie.insert.shuffle.parallelism", "4") - .option("hoodie.upsert.shuffle.parallelism", "4") - .option(DataSourceWriteOptions.RECORDKEY_FIELD.key, "key") - .option(DataSourceWriteOptions.PARTITIONPATH_FIELD.key, "nested_record.level") - .option(HoodieTableConfig.ORDERING_FIELDS.key, "ts") - .option(HoodieWriteConfig.TBL_NAME.key, "test_nested_partition") + // Commit 1 - Initial insert + val inputDF1 = spark.createDataFrame( + spark.sparkContext.parallelize(recordsCommit1), + schema + ) + inputDF1.write.format("hudi") + .options(writeOpts) .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL) .mode(SaveMode.Overwrite) .save(basePath) + val commit1 = DataSourceTestUtils.latestCommitCompletionTime(storage, basePath) + + // Commit 2 - Upsert: update key1 (int_field 1->100), insert key6 (INFO) + val recordsCommit2 = Seq( + Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), + Row("key6", 6L, "L6", 6, "str6", Row(60, "INFO")) + ) + val inputDF2 = spark.createDataFrame( + spark.sparkContext.parallelize(recordsCommit2), + schema + ) + inputDF2.write.format("hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val commit2 = DataSourceTestUtils.latestCommitCompletionTime(storage, basePath) + + // Commit 3 - Upsert: update key3 (int_field 3->300), insert key7 (INFO) + val recordsCommit3 = Seq( + Row("key3", 30L, "L3", 300, "str3", Row(30, "INFO")), + Row("key7", 7L, "L7", 7, "str7", Row(70, "INFO")) + ) + val inputDF3 = spark.createDataFrame( + spark.sparkContext.parallelize(recordsCommit3), + schema + ) + inputDF3.write.format("hudi") + .options(writeOpts) + .option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL) + .mode(SaveMode.Append) + .save(basePath) + val commit3 = DataSourceTestUtils.latestCommitCompletionTime(storage, basePath) + + // Snapshot read - filter on nested_record.level = 'INFO' (latest state: 5 records) + val snapshotResults = spark.read.format("hudi") + .load(basePath) + .filter("nested_record.level = 'INFO'") + .select("key", "ts", "level", "int_field", "string_field", "nested_record") + .orderBy("key") + .collect() + + val expectedSnapshot = Array( + Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), + Row("key3", 30L, "L3", 300, "str3", Row(30, "INFO")), + Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")), + Row("key6", 6L, "L6", 6, "str6", Row(60, "INFO")), + Row("key7", 7L, "L7", 7, "str7", Row(70, "INFO")) + ) + assertEquals(expectedSnapshot.length, snapshotResults.length, + s"Snapshot (INFO) count mismatch for $tableType") + expectedSnapshot.zip(snapshotResults).foreach { case (expected, actual) => + assertEquals(expected, actual) + } - // Read and filter on nested_record.level = 'INFO' - val results = spark.read.format("hudi") + // Time travel - as of commit1 (only initial 5 records; INFO = key1, key3, key5) + val timeTravelCommit1 = spark.read.format("hudi") + .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key, commit1) .load(basePath) .filter("nested_record.level = 'INFO'") .select("key", "ts", "level", "int_field", "string_field", "nested_record") .orderBy("key") .collect() - // Expected results - 3 records with nested_record.level = 'INFO' - val expectedResults = Array( + val expectedAfterCommit1 = Array( Row("key1", 1L, "L1", 1, "str1", Row(10, "INFO")), Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")) ) + assertEquals(expectedAfterCommit1.length, timeTravelCommit1.length, + s"Time travel to commit1 (INFO) count mismatch for $tableType") + expectedAfterCommit1.zip(timeTravelCommit1).foreach { case (expected, actual) => + assertEquals(expected, actual) + } + + // Time travel - as of commit2 (after 2nd commit; INFO = key1 updated, key3, key5, key6) + val timeTravelCommit2 = spark.read.format("hudi") + .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key, commit2) + .load(basePath) + .filter("nested_record.level = 'INFO'") + .select("key", "ts", "level", "int_field", "string_field", "nested_record") + .orderBy("key") + .collect() + + val expectedAfterCommit2 = Array( + Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), + Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), + Row("key5", 5L, "L5", 5, "str5", Row(50, "INFO")), + Row("key6", 6L, "L6", 6, "str6", Row(60, "INFO")) + ) + assertEquals(expectedAfterCommit2.length, timeTravelCommit2.length, + s"Time travel to commit2 (INFO) count mismatch for $tableType") + expectedAfterCommit2.zip(timeTravelCommit2).foreach { case (expected, actual) => + assertEquals(expected, actual) + } + + // Incremental query - from commit1 to commit2 (only key1 update and key6 insert; both INFO) + val incrementalCommit1To2 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.START_COMMIT.key, commit1) + .option(DataSourceReadOptions.END_COMMIT.key, commit2) + .load(basePath) + .filter("nested_record.level = 'INFO'") + .select("key", "ts", "level", "int_field", "string_field", "nested_record") + .orderBy("key") + .collect() + + val expectedInc1To2 = Array( + Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), + Row("key6", 6L, "L6", 6, "str6", Row(60, "INFO")) + ) + assertEquals(expectedInc1To2.length, incrementalCommit1To2.length, + s"Incremental (commit1->commit2, INFO) count mismatch for $tableType") + expectedInc1To2.zip(incrementalCommit1To2).foreach { case (expected, actual) => + assertEquals(expected, actual) + } - assertEquals(expectedResults.length, results.length) - expectedResults.zip(results).foreach { case (expected, actual) => + // Incremental query - from commit2 to commit3 (only key3 update and key7 insert; both INFO) + val incrementalCommit2To3 = spark.read.format("hudi") + .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) + .option(DataSourceReadOptions.START_COMMIT.key, commit2) + .option(DataSourceReadOptions.END_COMMIT.key, commit3) + .load(basePath) + .filter("nested_record.level = 'INFO'") + .select("key", "ts", "level", "int_field", "string_field", "nested_record") + .orderBy("key") + .collect() + + val expectedInc2To3 = Array( + Row("key3", 30L, "L3", 300, "str3", Row(30, "INFO")), + Row("key7", 7L, "L7", 7, "str7", Row(70, "INFO")) + ) + assertEquals(expectedInc2To3.length, incrementalCommit2To3.length, + s"Incremental (commit2->commit3, INFO) count mismatch for $tableType") + expectedInc2To3.zip(incrementalCommit2To3).foreach { case (expected, actual) => assertEquals(expected, actual) } } From ade116ed9a38e689220494437a9be170182a6865 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Sun, 15 Feb 2026 14:42:48 -0800 Subject: [PATCH 03/20] Addressed the comments --- .../hudi/SparkHoodieTableFileIndex.scala | 32 ++++++++++++++++--- .../hudi/functional/TestCOWDataSource.scala | 20 ++++++++---- .../hudi/functional/TestMORDataSource.scala | 5 +++ ...park3HoodiePruneFileSourcePartitions.scala | 24 ++++++++++++-- ...park4HoodiePruneFileSourcePartitions.scala | 22 ++++++++++++- 5 files changed, 89 insertions(+), 14 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index e4b79bcfe0842..2d0bf2a1f34e1 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -46,7 +46,7 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, InterpretedPredicate, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf @@ -241,10 +241,16 @@ class SparkHoodieTableFileIndex(spark: SparkSession, def listMatchingPartitionPaths(predicates: Seq[Expression]): Seq[PartitionPath] = { val resolve = spark.sessionState.analyzer.resolver val partitionColumnNames = getPartitionColumns + // Strip Spark's internal exprId suffix (e.g. #136) so nested_record#136 matches nested_record.level + def logicalRefName(ref: String): String = ref.replaceAll("#\\d+$", "") val partitionPruningPredicates = predicates.filter { _.references.map(_.name).forall { ref => - // NOTE: We're leveraging Spark's resolver here to appropriately handle case-sensitivity - partitionColumnNames.exists(partCol => resolve(ref, partCol)) + val logicalRef = logicalRefName(ref) + // NOTE: We're leveraging Spark's resolver here to appropriately handle case-sensitivity. + // For nested partition columns (e.g. nested_record.level), ref may be the struct root + // (e.g. nested_record#136); match when logicalRef equals partCol or is a prefix of partCol. + partitionColumnNames.exists(partCol => + resolve(logicalRef, partCol) || partCol.startsWith(logicalRef + ".")) } } @@ -272,10 +278,26 @@ class SparkHoodieTableFileIndex(spark: SparkSession, // the whole table if (haveProperPartitionValues(partitionPaths.toSeq) && partitionSchema.nonEmpty) { val predicate = partitionPruningPredicates.reduce(expressions.And) + val partitionFieldNames = partitionSchema.fieldNames + def getPartitionColumnPath(expr: Expression): Option[String] = expr match { + case a: AttributeReference => + Some(a.name.replaceAll("#\\d+$", "")) + case GetStructField(child, _, Some(fieldName)) => + getPartitionColumnPath(child).map(_ + "." + fieldName) + case _ => None + } val transformedPredicate = predicate.transform { + case g @ GetStructField(_, _, Some(_)) => + getPartitionColumnPath(g).flatMap { path => + val idx = partitionFieldNames.indexOf(path) + if (idx >= 0) Some(BoundReference(idx, partitionSchema(idx).dataType, nullable = true)) + else None + }.getOrElse(g) case a: AttributeReference => - val index = partitionSchema.indexWhere(a.name == _.name) - BoundReference(index, partitionSchema(index).dataType, nullable = true) + val logicalName = a.name.replaceAll("#\\d+$", "") + val index = partitionSchema.indexWhere(sf => resolve(logicalName, sf.name)) + if (index >= 0) BoundReference(index, partitionSchema(index).dataType, nullable = true) + else a } val boundPredicate: BasePredicate = try { // Try using 1-arg constructor via reflection diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 3eb44154ebf00..d20c092063eb5 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -42,7 +42,7 @@ import org.apache.hudi.hive.HiveSyncConfigHolder import org.apache.hudi.keygen.{ComplexKeyGenerator, CustomKeyGenerator, GlobalDeleteKeyGenerator, NonpartitionedKeyGenerator, SimpleKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.keygen.constant.{KeyGeneratorOptions, KeyGeneratorType} import org.apache.hudi.metrics.{Metrics, MetricsReporterType} -import org.apache.hudi.storage.{StoragePath, StoragePathFilter} +import org.apache.hudi.storage.{HoodieStorage, StoragePath, StoragePathFilter} import org.apache.hudi.table.HoodieSparkTable import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieSparkClientTestBase} import org.apache.hudi.util.JFunction @@ -2617,9 +2617,19 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals("value3", results(2).getAs[String]("data")) } - @ParameterizedTest - @CsvSource(Array("COW", "MOR")) - def testNestedFieldPartition(tableType: String): Unit = { + @Test + def testNestedFieldPartition(): Unit = { + TestCOWDataSource.runNestedFieldPartitionTest(spark, basePath, storage, "COW") + } +} + +object TestCOWDataSource { + + /** + * Shared test logic for nested field partition (COW and MOR). + * Used by TestCOWDataSource.testNestedFieldPartition and TestMORDataSource.testNestedFieldPartition. + */ + def runNestedFieldPartitionTest(spark: SparkSession, basePath: String, storage: HoodieStorage, tableType: String): Unit = { // Define schema with nested_record containing level field val nestedSchema = StructType(Seq( StructField("nested_int", IntegerType, nullable = false), @@ -2814,9 +2824,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup assertEquals(expected, actual) } } -} -object TestCOWDataSource { def convertColumnsToNullable(df: DataFrame, cols: String*): DataFrame = { cols.foldLeft(df) { (df, c) => // NOTE: This is the trick to make Spark convert a non-null column "c" into a nullable diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala index ab7bbcd097d28..81d049d432439 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestMORDataSource.scala @@ -2347,6 +2347,11 @@ class TestMORDataSource extends HoodieSparkClientTestBase with SparkDatasetMixin assertEquals("row4", results(3).getAs[String]("_row_key")) assertEquals("value4", results(3).getAs[String]("data")) } + + @Test + def testNestedFieldPartition(): Unit = { + TestCOWDataSource.runNestedFieldPartitionTest(spark, basePath, storage, "MOR") + } } object TestMORDataSource { diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index 589ee9774d3b7..b1d537c2fb6ec 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -36,7 +36,7 @@ import org.apache.spark.sql.types.StructType * Prune the partitions of Hudi table based relations by the means of pushing down the * partition filters * - * NOTE: [[HoodiePruneFileSourcePartitions]] is a replica in kind to Spark's [[PruneFileSourcePartitions]] + * NOTE: [[HoodiePruneFileSourcePartitions]] is a replica in kind to Spark's [[org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions]] */ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Rule[LogicalPlan] { @@ -105,11 +105,31 @@ private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } + /** + * Returns the logical name of an attribute by stripping Spark's internal exprId suffix (e.g. #136). + * Filter expressions may reference columns with these suffixed names (e.g. nested_record#136.level), + * while partition schema uses logical names (e.g. nested_record.level). + */ + private def logicalAttributeName(attr: AttributeReference): String = { + attr.name.replaceAll("#\\d+$", "") + } + + /** + * Returns true if the given attribute references a partition column. An attribute references a + * partition column if its logical name (without #exprId) equals a partition column name or + * is the struct parent of a nested partition path (e.g. nested_record for nested_record.level). + */ + private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { + val logicalName = logicalAttributeName(attr) + partitionSchema.names.contains(logicalName) || + partitionSchema.names.exists(_.startsWith(logicalName + ".")) + } + def getPartitionFiltersAndDataFilters(partitionSchema: StructType, normalizedFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { val partitionColumns = normalizedFilters.flatMap { expr => expr.collect { - case attr: AttributeReference if partitionSchema.names.contains(attr.name) => + case attr: AttributeReference if isPartitionColumnReference(attr, partitionSchema) => attr } } diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index 8412018c22db6..ecacf431bf585 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -105,11 +105,31 @@ private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } + /** + * Returns the logical name of an attribute by stripping Spark's internal exprId suffix (e.g. #136). + * Filter expressions may reference columns with these suffixed names (e.g. nested_record#136.level), + * while partition schema uses logical names (e.g. nested_record.level). + */ + private def logicalAttributeName(attr: AttributeReference): String = { + attr.name.replaceAll("#\\d+$", "") + } + + /** + * Returns true if the given attribute references a partition column. An attribute references a + * partition column if its logical name (without #exprId) equals a partition column name or + * is the struct parent of a nested partition path (e.g. nested_record for nested_record.level). + */ + private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { + val logicalName = logicalAttributeName(attr) + partitionSchema.names.contains(logicalName) || + partitionSchema.names.exists(_.startsWith(logicalName + ".")) + } + def getPartitionFiltersAndDataFilters(partitionSchema: StructType, normalizedFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { val partitionColumns = normalizedFilters.flatMap { expr => expr.collect { - case attr: AttributeReference if partitionSchema.names.contains(attr.name) => + case attr: AttributeReference if isPartitionColumnReference(attr, partitionSchema) => attr } } From 7c6392230868367358f2aa733419e0f98597b0e6 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Tue, 17 Feb 2026 15:02:30 -0800 Subject: [PATCH 04/20] Fix another bug for partition filter --- .../org/apache/hudi/HoodieFileIndex.scala | 96 +++++++++- .../org/apache/hudi/TestHoodieFileIndex.scala | 171 +++++++++++++++++- .../hudi/functional/TestCOWDataSource.scala | 97 +++++++++- 3 files changed, 354 insertions(+), 10 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 660592cf791ef..329d8354a17b8 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -37,7 +37,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable -import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.internal.SQLConf @@ -167,12 +167,25 @@ case class HoodieFileIndex(spark: SparkSession, /** * Invoked by Spark to fetch list of latest base files per partition. * - * @param partitionFilters partition column filters + * NOTE: For tables with nested partition columns (e.g. `nested_record.level`), Spark's + * [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] uses standard attribute-name + * matching when splitting filters into partition vs. data filters. Because the filter expression + * for `nested_record.level = 'INFO'` is represented as + * `GetStructField(AttributeReference("nested_record"), …)` — whose reference is the *struct* + * attribute `nested_record`, not the flat partition attribute `nested_record.level` — Spark + * classifies it as a data filter. This means `partitionFilters` arrives here empty and + * `dataFilters` contains the nested-field predicate. We re-split the combined set of filters + * below so that predicates whose only references are struct-parents of partition columns are + * treated as partition filters, matching the behaviour of [[HoodiePruneFileSourcePartitions]]. + * + * @param partitionFilters partition column filters (may be incomplete for nested columns) * @param dataFilters data columns filters * @return list of PartitionDirectory containing partition to base files mapping */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val slices = filterFileSlices(dataFilters, partitionFilters).flatMap( + val (actualPartitionFilters, actualDataFilters) = + reclassifyFiltersForNestedPartitionColumns(partitionFilters, dataFilters) + val slices = filterFileSlices(actualDataFilters, actualPartitionFilters).flatMap( { case (partitionOpt, fileSlices) => fileSlices.filter(!_.isEmpty).map(fs => ( InternalRow.fromSeq(partitionOpt.get.getValues), fs)) } @@ -180,6 +193,23 @@ case class HoodieFileIndex(spark: SparkSession, prepareFileSlices(slices) } + /** + * Re-splits the combined partition + data filters so that expressions whose attribute + * references are all struct-parents of nested partition columns (e.g. `nested_record` for + * partition column `nested_record.level`) are promoted to partition filters. + * + * This is a no-op when the partition schema contains no nested (dot-path) columns. + */ + private def reclassifyFiltersForNestedPartitionColumns( + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionSchema.fieldNames, + spark.sessionState.analyzer.resolver, + partitionFilters, + dataFilters) + } + protected def prepareFileSlices(slices: Seq[(InternalRow, FileSlice)]): Seq[PartitionDirectory] = { hasPushedDownPartitionPredicates = true @@ -502,6 +532,66 @@ object HoodieFileIndex extends Logging { val Strict: Val = Val("strict") } + /** + * Re-splits the combined set of filters so that predicates whose attribute references are + * all struct-parents of nested partition column names are promoted to partition filters. + * + * == Root cause of misclassification == + * In `FileSourceStrategy.apply` (Spark's physical planning rule), filters are split into + * partition filters and data filters using: + * {{{ + * val partitionSet = AttributeSet(l.resolve(relation.partitionSchema, resolver)) + * val (partitionFilters, dataFilters) = normalizedFilters.partition { f => + * f.references.subsetOf(partitionSet) + * } + * }}} + * For a Hudi table with partition column `nested_record.level`, the partition schema holds + * a flat `StructField("nested_record.level", StringType)`. However, when Spark's analyser + * resolves the user filter `nested_record.level = 'INFO'`, it sees `nested_record` as a + * known `StructType` attribute in the table output and rewrites the predicate as + * `GetStructField(AttributeReference("nested_record", StructType(…)), ordinal, "level") = "INFO"`. + * That expression's `references` set is `{nested_record}`. Because `{nested_record}` is not + * a subset of the `partitionSet` (which contains the unresolvable flat name `nested_record.level`), + * the predicate is classified as a data filter and `listFiles` is called with empty + * `partitionFilters`, bypassing partition pruning entirely. + * + * This method corrects the classification by treating any `AttributeReference` whose logical + * name is a struct-parent prefix of a nested partition column name (e.g. `nested_record` for + * `nested_record.level`) as a partition attribute, and re-partitions all filters accordingly. + * + * This is a no-op when `partitionColumnNames` contains no nested (dot-path) names. + * + * @param partitionColumnNames flat dot-path names of all partition columns (e.g. `["nested_record.level"]`) + * @param resolver case-sensitivity resolver from the active Spark session + * @param partitionFilters filters already classified as partition filters by Spark + * @param dataFilters filters already classified as data filters by Spark + * @return corrected `(partitionFilters, dataFilters)` pair + */ + private[hudi] def reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames: Seq[String], + resolver: (String, String) => Boolean, + partitionFilters: Seq[Expression], + dataFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { + // Only applies to tables that have at least one nested partition column. + if (!partitionColumnNames.exists(_.contains("."))) { + return (partitionFilters, dataFilters) + } + val allFilters = partitionFilters ++ dataFilters + // Identify AttributeReferences that are exact matches or struct-parent prefixes + // of nested partition column names (e.g. "nested_record" for "nested_record.level"). + val partitionAttrRefs = allFilters.flatMap { expr => + expr.collect { + case attr: AttributeReference + if { + val logicalName = attr.name.replaceAll("#\\d+$", "") + partitionColumnNames.exists(col => resolver(logicalName, col) || col.startsWith(logicalName + ".")) + } => attr + } + } + val partitionSet = AttributeSet(partitionAttrRefs) + allFilters.partition(f => f.references.subsetOf(partitionSet)) + } + def collectReferencedColumns(spark: SparkSession, queryFilters: Seq[Expression], schema: StructType): Seq[String] = { val resolver = spark.sessionState.analyzer.resolver val refs = queryFilters.flatMap(_.references) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index d5e0c6a927ac3..7635f3f13cba8 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -44,7 +44,7 @@ import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GetStructField, GreaterThanOrEqual, LessThan, Literal} import org.apache.spark.sql.execution.datasources.{NoopCache, PartitionDirectory} import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension @@ -858,6 +858,175 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS partitionValues.mkString(StoragePath.SEPARATOR) } } + + // ------------------------------------------------------------------------- + // Tests for HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns + // ------------------------------------------------------------------------- + + /** + * Build a simple GetStructField expression that mimics the Spark resolution of + * `structColName.fieldName = value` for the given struct schema. + * + * The struct is represented as a top-level AttributeReference; the field is accessed + * via GetStructField with the ordinal derived from `structSchema`. + */ + private def nestedEq(structColName: String, + structSchema: StructType, + fieldName: String, + value: String): EqualTo = { + val structAttr = AttributeReference(structColName, structSchema)() + val fieldOrdinal = structSchema.fieldIndex(fieldName) + EqualTo(GetStructField(structAttr, fieldOrdinal, Some(fieldName)), Literal(value)) + } + + /** A simple case-insensitive resolver (mirrors Spark's default). */ + private def resolver: (String, String) => Boolean = (a, b) => a.equalsIgnoreCase(b) + + @Test + def testReclassifyFlatPartitionColumnNoChange(): Unit = { + // Flat partition columns (no dots) → method should be a no-op. + val partitionColumnNames = Seq("dt") + val filter = EqualTo(AttributeReference("dt", StringType)(), Literal("2024-01-01")) + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(filter)) + // Nothing promoted: the caller already classified correctly for flat columns. + assertEquals(Seq.empty, pf) + assertEquals(Seq(filter), df) + } + + @Test + def testReclassifyEmptyPartitionColumnsNoChange(): Unit = { + // No partition columns at all → no-op. + val filter = EqualTo(AttributeReference("x", StringType)(), Literal("v")) + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames = Seq.empty, resolver, + partitionFilters = Seq.empty, dataFilters = Seq(filter)) + assertEquals(Seq.empty, pf) + assertEquals(Seq(filter), df) + } + + @Test + def testReclassifyNestedColumnPromotesGetStructFieldFromDataFilters(): Unit = { + // Core scenario: Spark puts GetStructField(nested_record, level) = 'INFO' into + // dataFilters because it can't match the struct attribute to the flat partition + // attribute "nested_record.level". The method must promote it to partitionFilters. + val nestedSchema = StructType(Seq( + StructField("nested_int", IntegerType), + StructField("level", StringType))) + val partitionColumnNames = Seq("nested_record.level") + val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(structFilter)) + + assertEquals(Seq(structFilter), pf, "GetStructField filter should be promoted to partition filters") + assertEquals(Seq.empty, df, "dataFilters should be empty after promotion") + } + + @Test + def testReclassifyNestedColumnAlreadyInPartitionFiltersNoChange(): Unit = { + // If Spark somehow already puts the filter in partitionFilters, the result stays the same. + val nestedSchema = StructType(Seq( + StructField("nested_int", IntegerType), + StructField("level", StringType))) + val partitionColumnNames = Seq("nested_record.level") + val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq(structFilter), dataFilters = Seq.empty) + + assertEquals(Seq(structFilter), pf) + assertEquals(Seq.empty, df) + } + + @Test + def testReclassifyMixedStructFilterPromotedDataFilterRetained(): Unit = { + // A filter on the struct-parent is promoted; a filter on an unrelated column stays. + val nestedSchema = StructType(Seq( + StructField("nested_int", IntegerType), + StructField("level", StringType))) + val partitionColumnNames = Seq("nested_record.level") + val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") + val tsFilter = GreaterThanOrEqual(AttributeReference("ts", LongType)(), Literal(1000L)) + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, + dataFilters = Seq(structFilter, tsFilter)) + + assertEquals(Seq(structFilter), pf, "Struct-field filter should be promoted") + assertEquals(Seq(tsFilter), df, "Unrelated data filter should remain in dataFilters") + } + + @Test + def testReclassifyAttrWithExprIdSuffixStrippedCorrectly(): Unit = { + // Spark sometimes appends an exprId suffix like "#136" to attribute names in filter + // expressions. The method must strip this suffix before comparing. + val nestedSchema = StructType(Seq(StructField("level", StringType))) + val partitionColumnNames = Seq("nested_record.level") + + // Simulate "nested_record#136" – the name produced by Spark's internal representation. + val structAttrWithSuffix = AttributeReference("nested_record#136", nestedSchema)() + val structFilter = EqualTo( + GetStructField(structAttrWithSuffix, 0, Some("level")), Literal("INFO")) + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(structFilter)) + + assertEquals(Seq(structFilter), pf, "Filter with exprId-suffixed attr name should be promoted") + assertEquals(Seq.empty, df) + } + + @Test + def testReclassifyMultipleNestedPartitionColumnsAllPromoted(): Unit = { + // Tables may have more than one nested partition column. + val schemaA = StructType(Seq(StructField("b", StringType))) + val schemaC = StructType(Seq(StructField("d", StringType))) + val partitionColumnNames = Seq("a.b", "c.d") + val filterAB = nestedEq("a", schemaA, "b", "x") + val filterCD = nestedEq("c", schemaC, "d", "y") + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, + dataFilters = Seq(filterAB, filterCD)) + + assertEquals(2, pf.size, "Both nested-column filters should be promoted") + assertEquals(Seq.empty, df) + } + + @Test + def testReclassifyFilterOnNonPartitionStructNotPromoted(): Unit = { + // A filter that references a struct that is NOT a partition-column parent + // must stay in dataFilters. + val otherSchema = StructType(Seq(StructField("field", StringType))) + val partitionColumnNames = Seq("nested_record.level") + val unrelatedFilter = nestedEq("other_struct", otherSchema, "field", "val") + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, + dataFilters = Seq(unrelatedFilter)) + + assertEquals(Seq.empty, pf, "Filter on an unrelated struct should not be promoted") + assertEquals(Seq(unrelatedFilter), df) + } + + @Test + def testReclassifyFilterReferencingBothPartitionAndDataNotPromoted(): Unit = { + // A conjunctive filter that references BOTH the partition struct-parent AND an + // unrelated column must not be promoted, because it also references data columns. + val nestedSchema = StructType(Seq(StructField("level", StringType))) + val partitionColumnNames = Seq("nested_record.level") + val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") + val tsFilter = GreaterThanOrEqual(AttributeReference("ts", LongType)(), Literal(1000L)) + // An AND combining partition-related and data columns: should stay in dataFilters. + val combined = And(structFilter, tsFilter) + + val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( + partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(combined)) + + assertEquals(Seq.empty, pf, + "Combined filter referencing both partition and data columns should not be promoted") + assertEquals(Seq(combined), df) + } } object TestHoodieFileIndex { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index d20c092063eb5..84470ef2c730b 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -2720,14 +2720,54 @@ object TestCOWDataSource { .save(basePath) val commit3 = DataSourceTestUtils.latestCommitCompletionTime(storage, basePath) + // Verify partition structure - we should have 3 partitions: INFO, ERROR, DEBUG + val allPartitions = storage.listDirectEntries(new StoragePath(basePath)) + .asScala.filter(_.isDirectory) + .map(_.getPath.getName) + .filterNot(_.startsWith(".")) // Filter out .hoodie and other hidden directories + .sorted + assertEquals(3, allPartitions.size, s"Expected 3 partitions for $tableType, but got: ${allPartitions.mkString(", ")}") + assertTrue(allPartitions.contains("INFO"), s"Missing INFO partition for $tableType") + assertTrue(allPartitions.contains("ERROR"), s"Missing ERROR partition for $tableType") + assertTrue(allPartitions.contains("DEBUG"), s"Missing DEBUG partition for $tableType") + // Snapshot read - filter on nested_record.level = 'INFO' (latest state: 5 records) - val snapshotResults = spark.read.format("hudi") + val snapshotDF = spark.read.format("hudi") .load(basePath) .filter("nested_record.level = 'INFO'") .select("key", "ts", "level", "int_field", "string_field", "nested_record") .orderBy("key") - .collect() + // VERIFICATION 1: Check partition schema contains the nested field + val snapshotRelation = snapshotDF.queryExecution.optimizedPlan.collectFirst { + case lr: org.apache.spark.sql.execution.datasources.LogicalRelation => lr + } + assertTrue(snapshotRelation.isDefined, s"LogicalRelation should exist for $tableType") + val fileIndex = snapshotRelation.get.relation match { + case fsRelation: org.apache.spark.sql.execution.datasources.HadoopFsRelation => + fsRelation.location.asInstanceOf[org.apache.hudi.HoodieFileIndex] + case baseRelation: org.apache.hudi.HoodieBaseRelation => + baseRelation.fileIndex + case _ => null + } + assertTrue(fileIndex != null, s"FileIndex should be available for $tableType") + assertEquals(1, fileIndex.partitionSchema.fields.length, + s"Partition schema should have 1 field for $tableType") + assertEquals("nested_record.level", fileIndex.partitionSchema.fields(0).name, + s"Partition field should be 'nested_record.level' for $tableType") + + // VERIFICATION 2: Check that predicates were pushed down to FileIndex + assertTrue(fileIndex.hasPredicatesPushedDown, + s"Partition predicates should be pushed down to FileIndex for $tableType") + + // VERIFICATION 3: Verify partition pruning by checking the physical plan + // The physical plan should show that only specific files are being scanned + val physicalPlan = snapshotDF.queryExecution.executedPlan.toString() + assertTrue(physicalPlan.contains("Scan") || physicalPlan.contains("FileScan"), + s"Physical plan should contain scan operation for $tableType") + + // Collect results to execute the query + val snapshotResults = snapshotDF.collect() val expectedSnapshot = Array( Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), Row("key3", 30L, "L3", 300, "str3", Row(30, "INFO")), @@ -2742,14 +2782,16 @@ object TestCOWDataSource { } // Time travel - as of commit1 (only initial 5 records; INFO = key1, key3, key5) - val timeTravelCommit1 = spark.read.format("hudi") + val timeTravelDF1 = spark.read.format("hudi") .option(DataSourceReadOptions.TIME_TRAVEL_AS_OF_INSTANT.key, commit1) .load(basePath) .filter("nested_record.level = 'INFO'") .select("key", "ts", "level", "int_field", "string_field", "nested_record") .orderBy("key") - .collect() + // VERIFICATION 4: Verify partition pruning works for time travel queries + // Check that the time travel query with partition filter returns correct results + val timeTravelCommit1 = timeTravelDF1.collect() val expectedAfterCommit1 = Array( Row("key1", 1L, "L1", 1, "str1", Row(10, "INFO")), Row("key3", 3L, "L3", 3, "str3", Row(30, "INFO")), @@ -2783,7 +2825,7 @@ object TestCOWDataSource { } // Incremental query - from commit1 to commit2 (only key1 update and key6 insert; both INFO) - val incrementalCommit1To2 = spark.read.format("hudi") + val incrementalDF1To2 = spark.read.format("hudi") .option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_INCREMENTAL_OPT_VAL) .option(DataSourceReadOptions.START_COMMIT.key, commit1) .option(DataSourceReadOptions.END_COMMIT.key, commit2) @@ -2791,8 +2833,15 @@ object TestCOWDataSource { .filter("nested_record.level = 'INFO'") .select("key", "ts", "level", "int_field", "string_field", "nested_record") .orderBy("key") - .collect() + // VERIFICATION 6: Verify partition filtering works for incremental queries + // For incremental queries, the filter on nested_record.level should still limit scanned data + val incrementalPlan1To2 = incrementalDF1To2.queryExecution.executedPlan.toString() + // The plan should show filtering is happening + assertTrue(incrementalPlan1To2.contains("Filter") || incrementalPlan1To2.contains("Scan"), + s"Incremental query plan should show filtering for $tableType") + + val incrementalCommit1To2 = incrementalDF1To2.collect() val expectedInc1To2 = Array( Row("key1", 10L, "L1", 100, "str1", Row(10, "INFO")), Row("key6", 6L, "L6", 6, "str6", Row(60, "INFO")) @@ -2823,6 +2872,42 @@ object TestCOWDataSource { expectedInc2To3.zip(incrementalCommit2To3).foreach { case (expected, actual) => assertEquals(expected, actual) } + + // VERIFICATION 4: Test with different partition values to ensure filtering is working correctly + // Query for ERROR partition (should only return key2) + val errorPartitionDF = spark.read.format("hudi") + .load(basePath) + .filter("nested_record.level = 'ERROR'") + .select("key", "nested_record") + + val errorResults = errorPartitionDF.collect() + assertEquals(1, errorResults.length, s"ERROR partition should have 1 record for $tableType") + assertEquals("key2", errorResults(0).getString(0), + s"ERROR partition should contain key2 for $tableType") + + // VERIFICATION 5: Test with DEBUG partition + val debugPartitionDF = spark.read.format("hudi") + .load(basePath) + .filter("nested_record.level = 'DEBUG'") + .select("key", "nested_record") + + val debugResults = debugPartitionDF.collect() + assertEquals(1, debugResults.length, s"DEBUG partition should have 1 record for $tableType") + assertEquals("key4", debugResults(0).getString(0), + s"DEBUG partition should contain key4 for $tableType") + + // VERIFICATION 6: Verify that filtering on top-level 'level' field returns correct results + // This ensures we're correctly distinguishing between nested_record.level (partition) and level (data column) + val topLevelFilterDF = spark.read.format("hudi") + .load(basePath) + .filter("level = 'L1'") // Filter on top-level 'level', not nested_record.level + .select("key", "level", "nested_record") + + val topLevelResults = topLevelFilterDF.collect() + // Should return key1 which has level='L1' and is in INFO partition + assertEquals(1, topLevelResults.length, s"Top-level level='L1' should return 1 record for $tableType") + assertEquals("key1", topLevelResults(0).getString(0), + s"Top-level level='L1' should return key1 for $tableType") } def convertColumnsToNullable(df: DataFrame, cols: String*): DataFrame = { From 7dbec4b830490783e8ddf0541c4ad4ff7488b8c4 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Mon, 30 Mar 2026 12:23:40 -0700 Subject: [PATCH 05/20] remove reclassification --- .../org/apache/hudi/HoodieFileIndex.scala | 133 ++++---------- .../hudi/SparkHoodieTableFileIndex.scala | 111 +++++++++++- .../org/apache/hudi/TestHoodieFileIndex.scala | 170 +----------------- ...park3HoodiePruneFileSourcePartitions.scala | 2 +- ...park4HoodiePruneFileSourcePartitions.scala | 2 +- 5 files changed, 147 insertions(+), 271 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 329d8354a17b8..9a2efa86e46fd 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -37,7 +37,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.internal.SQLConf @@ -105,6 +105,15 @@ case class HoodieFileIndex(spark: SparkSession, @transient protected var hasPushedDownPartitionPredicates: Boolean = false + /** + * Partition-pruned file slices cached by [[HoodiePruneFileSourcePartitions]] (called before + * [[listFiles]]). Non-null only when the optimizer rule has already run. Used in + * [[listFiles]] to avoid re-running partition pruning from scratch for tables with nested + * partition columns where Spark's [[FileSourceScanExec]] passes empty `partitionFilters` + * (because the flat dot-path partition schema cannot match [[GetStructField]] expressions). + */ + @transient private var cachedPrunedSlices: Option[Seq[(Option[PartitionPath], Seq[FileSlice])]] = None + /** * NOTE: [[indicesSupport]] is a transient state, since it's only relevant while logical plan * is handled by the Spark's driver @@ -167,49 +176,29 @@ case class HoodieFileIndex(spark: SparkSession, /** * Invoked by Spark to fetch list of latest base files per partition. * - * NOTE: For tables with nested partition columns (e.g. `nested_record.level`), Spark's - * [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] uses standard attribute-name - * matching when splitting filters into partition vs. data filters. Because the filter expression - * for `nested_record.level = 'INFO'` is represented as - * `GetStructField(AttributeReference("nested_record"), …)` — whose reference is the *struct* - * attribute `nested_record`, not the flat partition attribute `nested_record.level` — Spark - * classifies it as a data filter. This means `partitionFilters` arrives here empty and - * `dataFilters` contains the nested-field predicate. We re-split the combined set of filters - * below so that predicates whose only references are struct-parents of partition columns are - * treated as partition filters, matching the behaviour of [[HoodiePruneFileSourcePartitions]]. - * - * @param partitionFilters partition column filters (may be incomplete for nested columns) - * @param dataFilters data columns filters - * @return list of PartitionDirectory containing partition to base files mapping + * When [[HoodiePruneFileSourcePartitions]] has already pushed down partition predicates and + * cached partition-pruned file slices, those cached slices are reused directly. This is + * required for tables with nested partition columns (e.g. `nested_record.level`): because + * [[HadoopFsRelation]] exposes a flat dot-path partition schema, Spark's + * [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] cannot match + * [[GetStructField]]-based filter references against it and therefore passes empty + * `partitionFilters` here — which would cause all partitions to be scanned. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val (actualPartitionFilters, actualDataFilters) = - reclassifyFiltersForNestedPartitionColumns(partitionFilters, dataFilters) - val slices = filterFileSlices(actualDataFilters, actualPartitionFilters).flatMap( - { case (partitionOpt, fileSlices) => - fileSlices.filter(!_.isEmpty).map(fs => ( InternalRow.fromSeq(partitionOpt.get.getValues), fs)) + val slices = if (hasPushedDownPartitionPredicates && cachedPrunedSlices.isDefined) { + cachedPrunedSlices.get.flatMap { case (partitionOpt, fileSlices) => + fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) } - ) + } else { + filterFileSlices(dataFilters, partitionFilters).flatMap( + { case (partitionOpt, fileSlices) => + fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) + } + ) + } prepareFileSlices(slices) } - /** - * Re-splits the combined partition + data filters so that expressions whose attribute - * references are all struct-parents of nested partition columns (e.g. `nested_record` for - * partition column `nested_record.level`) are promoted to partition filters. - * - * This is a no-op when the partition schema contains no nested (dot-path) columns. - */ - private def reclassifyFiltersForNestedPartitionColumns( - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionSchema.fieldNames, - spark.sessionState.analyzer.resolver, - partitionFilters, - dataFilters) - } - protected def prepareFileSlices(slices: Seq[(InternalRow, FileSlice)]): Seq[PartitionDirectory] = { hasPushedDownPartitionPredicates = true @@ -257,8 +246,14 @@ case class HoodieFileIndex(spark: SparkSession, prunePartitionsAndGetFileSlices(dataFilters, partitionFilters) hasPushedDownPartitionPredicates = true + // Cache partition-pruned slices so that the subsequent listFiles call (from + // FileSourceScanExec) can reuse them for tables with nested partition columns. + if (isPartitionPruned) { + cachedPrunedSlices = Some(prunedPartitionsAndFileSlices) + } + // If there are no data filters, return all the file slices. - // If isPartitionPurge is true, this fun is trigger by HoodiePruneFileSourcePartitions, don't look up candidate files + // If isPartitionPruned is true, this fun is triggered by HoodiePruneFileSourcePartitions, don't look up candidate files // If there are no file slices, return empty list. if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty || isPartitionPruned ) { prunedPartitionsAndFileSlices @@ -532,66 +527,6 @@ object HoodieFileIndex extends Logging { val Strict: Val = Val("strict") } - /** - * Re-splits the combined set of filters so that predicates whose attribute references are - * all struct-parents of nested partition column names are promoted to partition filters. - * - * == Root cause of misclassification == - * In `FileSourceStrategy.apply` (Spark's physical planning rule), filters are split into - * partition filters and data filters using: - * {{{ - * val partitionSet = AttributeSet(l.resolve(relation.partitionSchema, resolver)) - * val (partitionFilters, dataFilters) = normalizedFilters.partition { f => - * f.references.subsetOf(partitionSet) - * } - * }}} - * For a Hudi table with partition column `nested_record.level`, the partition schema holds - * a flat `StructField("nested_record.level", StringType)`. However, when Spark's analyser - * resolves the user filter `nested_record.level = 'INFO'`, it sees `nested_record` as a - * known `StructType` attribute in the table output and rewrites the predicate as - * `GetStructField(AttributeReference("nested_record", StructType(…)), ordinal, "level") = "INFO"`. - * That expression's `references` set is `{nested_record}`. Because `{nested_record}` is not - * a subset of the `partitionSet` (which contains the unresolvable flat name `nested_record.level`), - * the predicate is classified as a data filter and `listFiles` is called with empty - * `partitionFilters`, bypassing partition pruning entirely. - * - * This method corrects the classification by treating any `AttributeReference` whose logical - * name is a struct-parent prefix of a nested partition column name (e.g. `nested_record` for - * `nested_record.level`) as a partition attribute, and re-partitions all filters accordingly. - * - * This is a no-op when `partitionColumnNames` contains no nested (dot-path) names. - * - * @param partitionColumnNames flat dot-path names of all partition columns (e.g. `["nested_record.level"]`) - * @param resolver case-sensitivity resolver from the active Spark session - * @param partitionFilters filters already classified as partition filters by Spark - * @param dataFilters filters already classified as data filters by Spark - * @return corrected `(partitionFilters, dataFilters)` pair - */ - private[hudi] def reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames: Seq[String], - resolver: (String, String) => Boolean, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - // Only applies to tables that have at least one nested partition column. - if (!partitionColumnNames.exists(_.contains("."))) { - return (partitionFilters, dataFilters) - } - val allFilters = partitionFilters ++ dataFilters - // Identify AttributeReferences that are exact matches or struct-parent prefixes - // of nested partition column names (e.g. "nested_record" for "nested_record.level"). - val partitionAttrRefs = allFilters.flatMap { expr => - expr.collect { - case attr: AttributeReference - if { - val logicalName = attr.name.replaceAll("#\\d+$", "") - partitionColumnNames.exists(col => resolver(logicalName, col) || col.startsWith(logicalName + ".")) - } => attr - } - } - val partitionSet = AttributeSet(partitionAttrRefs) - allFilters.partition(f => f.references.subsetOf(partitionSet)) - } - def collectReferencedColumns(spark: SparkSession, queryFilters: Seq[Expression], schema: StructType): Seq[String] = { val resolver = spark.sessionState.analyzer.resolver val refs = queryFilters.flatMap(_.references) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 2d0bf2a1f34e1..6fdaade69aa5f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -46,7 +46,7 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, IsNotNull, IsNull, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf @@ -204,6 +204,25 @@ class SparkHoodieTableFileIndex(spark: SparkSession, } } + /** + * Spark-facing partition schema that preserves nested structure for nested partition columns. + * + * NOTE: Hudi's [[partitionSchema]] intentionally returns a *flat* schema where field names use full + * dot-paths (for example, "a.b.c") to avoid collisions with top-level data columns. Some Spark + * planner/analyzer paths, however, reason about nested columns as nested [[StructType]]s and + * require a nested schema shape to properly resolve [[GetStructField]] chains. + * + * This method reconstructs a nested [[StructType]] from the flat partition schema, using the same + * leaf data-types, and preserving deterministic field ordering based on the original flat schema. + */ + def partitionSchemaForSpark: StructType = { + if (!shouldReadAsPartitionedTable) { + new StructType() + } else { + SparkHoodieTableFileIndex.buildNestedPartitionSchema(_partitionSchemaFromProperties) + } + } + /** * Fetch list of latest base files w/ corresponding log files, after performing * partition pruning @@ -293,6 +312,26 @@ class SparkHoodieTableFileIndex(spark: SparkSession, if (idx >= 0) Some(BoundReference(idx, partitionSchema(idx).dataType, nullable = true)) else None }.getOrElse(g) + // NOTE: Spark's optimizer auto-adds IsNotNull(struct_attr) when a nested field is + // filtered (e.g. IsNotNull(nested_record) for filter nested_record.level = 'INFO'). + // The struct attribute cannot be directly bound to a flat partition value. + // Since partition rows are parsed from partition paths they are never null, so + // IsNotNull for any expression whose references are all struct-parents of nested + // partition columns is always true (and IsNull always false). + // We match on the expression's references rather than requiring the child to be a + // plain AttributeReference, because the optimizer may wrap it in casts or other nodes. + case n @ IsNotNull(_) + if n.references.map(_.name).nonEmpty && + n.references.map(_.name).forall { ref => + val logicalName = ref.replaceAll("#\\d+$", "") + partitionFieldNames.exists(_.startsWith(logicalName + ".")) + } => Literal(true) + case n @ IsNull(_) + if n.references.map(_.name).nonEmpty && + n.references.map(_.name).forall { ref => + val logicalName = ref.replaceAll("#\\d+$", "") + partitionFieldNames.exists(_.startsWith(logicalName + ".")) + } => Literal(false) case a: AttributeReference => val logicalName = a.name.replaceAll("#\\d+$", "") val index = partitionSchema.indexWhere(sf => resolve(logicalName, sf.name)) @@ -513,6 +552,76 @@ object SparkHoodieTableFileIndex extends SparkAdapterSupport { private val LOG = LoggerFactory.getLogger(classOf[SparkHoodieTableFileIndex]) private val PUT_LEAF_FILES_METHOD_NAME = "putLeafFiles" + private case class NestedFieldNode( + leafType: Option[org.apache.spark.sql.types.DataType], + children: scala.collection.mutable.LinkedHashMap[String, NestedFieldNode] + ) + + /** + * Reconstruct nested partition schema from a flat partition schema containing dot-path field names. + * + * For example, flat fields ["a.b": int, "a.c": string, "d": long] becomes: + * + * StructType( + * StructField("a", StructType(StructField("b", int), StructField("c", string))), + * StructField("d", long) + * ) + */ + private[hudi] def buildNestedPartitionSchema(flatPartitionSchema: StructType): StructType = { + if (flatPartitionSchema.isEmpty) { + new StructType() + } else { + val root = NestedFieldNode(None, scala.collection.mutable.LinkedHashMap.empty) + + def getOrCreateChild(parent: NestedFieldNode, name: String): NestedFieldNode = { + parent.children.getOrElseUpdate(name, NestedFieldNode(None, scala.collection.mutable.LinkedHashMap.empty)) + } + + flatPartitionSchema.fields.foreach { field => + val parts = field.name.split("\\.", -1) + checkState(parts.forall(p => p.nonEmpty), + s"Invalid partition field path '${field.name}' in partition schema") + + var node = root + var i = 0 + while (i < parts.length) { + val part = parts(i) + val isLeaf = i == parts.length - 1 + + if (isLeaf) { + val child = getOrCreateChild(node, part) + checkState(child.children.isEmpty, + s"Conflicting partition schema: '${field.name}' collides with nested fields under '${parts.take(i + 1).mkString(".")}'") + checkState(child.leafType.isEmpty || child.leafType.contains(field.dataType), + s"Conflicting partition schema: '${field.name}' has inconsistent types (${child.leafType.orNull} vs ${field.dataType})") + node.children.update(part, child.copy(leafType = Some(field.dataType))) + } else { + val child = getOrCreateChild(node, part) + checkState(child.leafType.isEmpty, + s"Conflicting partition schema: '${field.name}' requires struct at '${parts.take(i + 1).mkString(".")}', but a leaf is defined") + node = child + } + + i += 1 + } + } + + def toStructType(node: NestedFieldNode): StructType = { + val fields = node.children.map { case (name, child) => + child.leafType match { + case Some(dt) if child.children.isEmpty => + StructField(name, dt, nullable = true) + case _ => + StructField(name, toStructType(child), nullable = true) + } + }.toArray + StructType(fields) + } + + toStructType(root) + } + } + private def haveProperPartitionValues(partitionPaths: Seq[PartitionPath]) = { partitionPaths.forall(_.getValues.length > 0) } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 7635f3f13cba8..7c6bff0be08f6 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -44,7 +44,7 @@ import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GetStructField, GreaterThanOrEqual, LessThan, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal} import org.apache.spark.sql.execution.datasources.{NoopCache, PartitionDirectory} import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension @@ -859,174 +859,6 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } } - // ------------------------------------------------------------------------- - // Tests for HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns - // ------------------------------------------------------------------------- - - /** - * Build a simple GetStructField expression that mimics the Spark resolution of - * `structColName.fieldName = value` for the given struct schema. - * - * The struct is represented as a top-level AttributeReference; the field is accessed - * via GetStructField with the ordinal derived from `structSchema`. - */ - private def nestedEq(structColName: String, - structSchema: StructType, - fieldName: String, - value: String): EqualTo = { - val structAttr = AttributeReference(structColName, structSchema)() - val fieldOrdinal = structSchema.fieldIndex(fieldName) - EqualTo(GetStructField(structAttr, fieldOrdinal, Some(fieldName)), Literal(value)) - } - - /** A simple case-insensitive resolver (mirrors Spark's default). */ - private def resolver: (String, String) => Boolean = (a, b) => a.equalsIgnoreCase(b) - - @Test - def testReclassifyFlatPartitionColumnNoChange(): Unit = { - // Flat partition columns (no dots) → method should be a no-op. - val partitionColumnNames = Seq("dt") - val filter = EqualTo(AttributeReference("dt", StringType)(), Literal("2024-01-01")) - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(filter)) - // Nothing promoted: the caller already classified correctly for flat columns. - assertEquals(Seq.empty, pf) - assertEquals(Seq(filter), df) - } - - @Test - def testReclassifyEmptyPartitionColumnsNoChange(): Unit = { - // No partition columns at all → no-op. - val filter = EqualTo(AttributeReference("x", StringType)(), Literal("v")) - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames = Seq.empty, resolver, - partitionFilters = Seq.empty, dataFilters = Seq(filter)) - assertEquals(Seq.empty, pf) - assertEquals(Seq(filter), df) - } - - @Test - def testReclassifyNestedColumnPromotesGetStructFieldFromDataFilters(): Unit = { - // Core scenario: Spark puts GetStructField(nested_record, level) = 'INFO' into - // dataFilters because it can't match the struct attribute to the flat partition - // attribute "nested_record.level". The method must promote it to partitionFilters. - val nestedSchema = StructType(Seq( - StructField("nested_int", IntegerType), - StructField("level", StringType))) - val partitionColumnNames = Seq("nested_record.level") - val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(structFilter)) - - assertEquals(Seq(structFilter), pf, "GetStructField filter should be promoted to partition filters") - assertEquals(Seq.empty, df, "dataFilters should be empty after promotion") - } - - @Test - def testReclassifyNestedColumnAlreadyInPartitionFiltersNoChange(): Unit = { - // If Spark somehow already puts the filter in partitionFilters, the result stays the same. - val nestedSchema = StructType(Seq( - StructField("nested_int", IntegerType), - StructField("level", StringType))) - val partitionColumnNames = Seq("nested_record.level") - val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq(structFilter), dataFilters = Seq.empty) - - assertEquals(Seq(structFilter), pf) - assertEquals(Seq.empty, df) - } - - @Test - def testReclassifyMixedStructFilterPromotedDataFilterRetained(): Unit = { - // A filter on the struct-parent is promoted; a filter on an unrelated column stays. - val nestedSchema = StructType(Seq( - StructField("nested_int", IntegerType), - StructField("level", StringType))) - val partitionColumnNames = Seq("nested_record.level") - val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") - val tsFilter = GreaterThanOrEqual(AttributeReference("ts", LongType)(), Literal(1000L)) - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, - dataFilters = Seq(structFilter, tsFilter)) - - assertEquals(Seq(structFilter), pf, "Struct-field filter should be promoted") - assertEquals(Seq(tsFilter), df, "Unrelated data filter should remain in dataFilters") - } - - @Test - def testReclassifyAttrWithExprIdSuffixStrippedCorrectly(): Unit = { - // Spark sometimes appends an exprId suffix like "#136" to attribute names in filter - // expressions. The method must strip this suffix before comparing. - val nestedSchema = StructType(Seq(StructField("level", StringType))) - val partitionColumnNames = Seq("nested_record.level") - - // Simulate "nested_record#136" – the name produced by Spark's internal representation. - val structAttrWithSuffix = AttributeReference("nested_record#136", nestedSchema)() - val structFilter = EqualTo( - GetStructField(structAttrWithSuffix, 0, Some("level")), Literal("INFO")) - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(structFilter)) - - assertEquals(Seq(structFilter), pf, "Filter with exprId-suffixed attr name should be promoted") - assertEquals(Seq.empty, df) - } - - @Test - def testReclassifyMultipleNestedPartitionColumnsAllPromoted(): Unit = { - // Tables may have more than one nested partition column. - val schemaA = StructType(Seq(StructField("b", StringType))) - val schemaC = StructType(Seq(StructField("d", StringType))) - val partitionColumnNames = Seq("a.b", "c.d") - val filterAB = nestedEq("a", schemaA, "b", "x") - val filterCD = nestedEq("c", schemaC, "d", "y") - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, - dataFilters = Seq(filterAB, filterCD)) - - assertEquals(2, pf.size, "Both nested-column filters should be promoted") - assertEquals(Seq.empty, df) - } - - @Test - def testReclassifyFilterOnNonPartitionStructNotPromoted(): Unit = { - // A filter that references a struct that is NOT a partition-column parent - // must stay in dataFilters. - val otherSchema = StructType(Seq(StructField("field", StringType))) - val partitionColumnNames = Seq("nested_record.level") - val unrelatedFilter = nestedEq("other_struct", otherSchema, "field", "val") - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, - dataFilters = Seq(unrelatedFilter)) - - assertEquals(Seq.empty, pf, "Filter on an unrelated struct should not be promoted") - assertEquals(Seq(unrelatedFilter), df) - } - - @Test - def testReclassifyFilterReferencingBothPartitionAndDataNotPromoted(): Unit = { - // A conjunctive filter that references BOTH the partition struct-parent AND an - // unrelated column must not be promoted, because it also references data columns. - val nestedSchema = StructType(Seq(StructField("level", StringType))) - val partitionColumnNames = Seq("nested_record.level") - val structFilter = nestedEq("nested_record", nestedSchema, "level", "INFO") - val tsFilter = GreaterThanOrEqual(AttributeReference("ts", LongType)(), Literal(1000L)) - // An AND combining partition-related and data columns: should stay in dataFilters. - val combined = And(structFilter, tsFilter) - - val (pf, df) = HoodieFileIndex.reclassifyFiltersForNestedPartitionColumns( - partitionColumnNames, resolver, partitionFilters = Seq.empty, dataFilters = Seq(combined)) - - assertEquals(Seq.empty, pf, - "Combined filter referencing both partition and data columns should not be promoted") - assertEquals(Seq(combined), df) - } } object TestHoodieFileIndex { diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index b1d537c2fb6ec..aac93d6202512 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -48,7 +48,7 @@ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru val normalizedFilters = exprUtils.normalizeExprs(deterministicFilters, lr.output) val (partitionPruningFilters, dataFilters) = - getPartitionFiltersAndDataFilters(fileIndex.partitionSchema, normalizedFilters) + getPartitionFiltersAndDataFilters(fileIndex.partitionSchemaForSpark, normalizedFilters) // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index ecacf431bf585..079b7b1ec8c84 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -48,7 +48,7 @@ case class Spark4HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru val normalizedFilters = exprUtils.normalizeExprs(deterministicFilters, lr.output) val (partitionPruningFilters, dataFilters) = - getPartitionFiltersAndDataFilters(fileIndex.partitionSchema, normalizedFilters) + getPartitionFiltersAndDataFilters(fileIndex.partitionSchemaForSpark, normalizedFilters) // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats From 9ed64961c249a8d8de372d6681be2b155b88ce52 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Tue, 31 Mar 2026 15:50:28 -0700 Subject: [PATCH 06/20] fix CI failures --- .../org/apache/hudi/HoodieFileIndex.scala | 38 +++++++++++++------ 1 file changed, 26 insertions(+), 12 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 9a2efa86e46fd..d9ee587e03cc7 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -179,13 +179,14 @@ case class HoodieFileIndex(spark: SparkSession, * When [[HoodiePruneFileSourcePartitions]] has already pushed down partition predicates and * cached partition-pruned file slices, those cached slices are reused directly. This is * required for tables with nested partition columns (e.g. `nested_record.level`): because - * [[HadoopFsRelation]] exposes a flat dot-path partition schema, Spark's + * [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]] exposes a flat dot-path partition schema, Spark's * [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] cannot match - * [[GetStructField]]-based filter references against it and therefore passes empty + * [[org.apache.spark.sql.catalyst.expressions.GetStructField]]-based filter references against it and therefore passes empty * `partitionFilters` here — which would cause all partitions to be scanned. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val slices = if (hasPushedDownPartitionPredicates && cachedPrunedSlices.isDefined) { + val usedCache = hasPushedDownPartitionPredicates && cachedPrunedSlices.isDefined + val slices = if (usedCache) { cachedPrunedSlices.get.flatMap { case (partitionOpt, fileSlices) => fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) } @@ -196,7 +197,17 @@ case class HoodieFileIndex(spark: SparkSession, } ) } - prepareFileSlices(slices) + val result = prepareFileSlices(slices) + if (usedCache) { + // Reset so that if this file-index instance is reused for a subsequent query execution + // (possible because HoodieTableMetaClient.equals is content-based, causing Spark's optimizer + // to treat two independently-created file indexes for the same table as equivalent and reuse + // one plan for both), HoodiePruneFileSourcePartitions will re-run and repopulate the cache + // with a fresh read of the table state rather than serving stale pre-mutation data. + hasPushedDownPartitionPredicates = false + cachedPrunedSlices = None + } + result } protected def prepareFileSlices(slices: Seq[(InternalRow, FileSlice)]): Seq[PartitionDirectory] = { @@ -246,16 +257,9 @@ case class HoodieFileIndex(spark: SparkSession, prunePartitionsAndGetFileSlices(dataFilters, partitionFilters) hasPushedDownPartitionPredicates = true - // Cache partition-pruned slices so that the subsequent listFiles call (from - // FileSourceScanExec) can reuse them for tables with nested partition columns. - if (isPartitionPruned) { - cachedPrunedSlices = Some(prunedPartitionsAndFileSlices) - } - // If there are no data filters, return all the file slices. - // If isPartitionPruned is true, this fun is triggered by HoodiePruneFileSourcePartitions, don't look up candidate files // If there are no file slices, return empty list. - if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty || isPartitionPruned ) { + val result = if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty) { prunedPartitionsAndFileSlices } else { // Look up candidate files names in the col-stats or record level index, if all of the following conditions are true @@ -316,6 +320,16 @@ case class HoodieFileIndex(spark: SparkSession, prunedPartitionsAndFilteredFileSlices } + + // Cache the post-data-skipping slices so that the subsequent listFiles call (from + // FileSourceScanExec) can reuse them for tables with nested partition columns. + // The cache must be populated after data skipping so that listFiles does not re-read + // files that were already eliminated by column-stats. + if (isPartitionPruned) { + cachedPrunedSlices = Some(result) + } + + result } /** From af1aa0ff5766b4767558bf91bcd8cb53f1985d9b Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Tue, 31 Mar 2026 18:36:54 -0700 Subject: [PATCH 07/20] fix CI issues --- .../org/apache/hudi/HoodieFileIndex.scala | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index d9ee587e03cc7..d53217574c688 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -185,12 +185,28 @@ case class HoodieFileIndex(spark: SparkSession, * `partitionFilters` here — which would cause all partitions to be scanned. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - val usedCache = hasPushedDownPartitionPredicates && cachedPrunedSlices.isDefined + // Use the cached result from HoodiePruneFileSourcePartitions ONLY when FileSourceScanExec + // passes empty partitionFilters. This is the signature of the nested-partition-column case: + // because HadoopFsRelation exposes a flat dot-path partition schema, FileSourceScanExec + // cannot match GetStructField-based filter references and therefore passes partitionFilters=[] + // here, which would cause all partitions to be scanned without the cache. + // + // For regular partition columns FileSourceScanExec always passes non-empty partitionFilters, + // so we call filterFileSlices fresh regardless of the cache. This is critical for correctness: + // the cache may have been populated by a *different* query's HoodiePruneFileSourcePartitions + // run (e.g. from a .executedPlan.toString() call that triggers planning but not execution, + // leaving hasPushedDownPartitionPredicates=true with stale slices), and using it would return + // wrong rows for a query with different filters. + val usedCache = partitionFilters.isEmpty && cachedPrunedSlices.isDefined val slices = if (usedCache) { cachedPrunedSlices.get.flatMap { case (partitionOpt, fileSlices) => fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) } } else { + // A non-empty partitionFilters means we are on the regular-partition path and must compute + // fresh results. Discard any stale cache so it cannot be picked up by a later + // nested-partition query on the same file-index instance. + cachedPrunedSlices = None filterFileSlices(dataFilters, partitionFilters).flatMap( { case (partitionOpt, fileSlices) => fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) From 874aa25d95ac7648094ec1cdbeff6b5bd545ad35 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 12:34:13 -0700 Subject: [PATCH 08/20] refactor --- .../org/apache/hudi/HoodieFileIndex.scala | 93 +++++++------------ ...park3HoodiePruneFileSourcePartitions.scala | 2 +- ...ark33HoodiePruneFileSourcePartitions.scala | 2 +- ...park4HoodiePruneFileSourcePartitions.scala | 2 +- 4 files changed, 36 insertions(+), 63 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index d53217574c688..ba1eed8a1fbb5 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -106,13 +106,14 @@ case class HoodieFileIndex(spark: SparkSession, @transient protected var hasPushedDownPartitionPredicates: Boolean = false /** - * Partition-pruned file slices cached by [[HoodiePruneFileSourcePartitions]] (called before - * [[listFiles]]). Non-null only when the optimizer rule has already run. Used in - * [[listFiles]] to avoid re-running partition pruning from scratch for tables with nested - * partition columns where Spark's [[FileSourceScanExec]] passes empty `partitionFilters` - * (because the flat dot-path partition schema cannot match [[GetStructField]] expressions). + * True when any partition column name contains a dot, indicating a nested field path + * (e.g. "nested_record.level"). For such columns, Spark's [[FileSourceScanExec]] cannot + * match [[GetStructField]]-based filter references against the flat dot-path partition schema + * exposed by [[HadoopFsRelation]], and therefore passes empty `partitionFilters` to + * [[listFiles]]. We detect this and re-extract partition predicates from `dataFilters`. */ - @transient private var cachedPrunedSlices: Option[Seq[(Option[PartitionPath], Seq[FileSlice])]] = None + private val hasNestedPartitionColumns: Boolean = + getPartitionColumns.exists(_.contains(".")) /** * NOTE: [[indicesSupport]] is a transient state, since it's only relevant while logical plan @@ -176,54 +177,35 @@ case class HoodieFileIndex(spark: SparkSession, /** * Invoked by Spark to fetch list of latest base files per partition. * - * When [[HoodiePruneFileSourcePartitions]] has already pushed down partition predicates and - * cached partition-pruned file slices, those cached slices are reused directly. This is - * required for tables with nested partition columns (e.g. `nested_record.level`): because - * [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]] exposes a flat dot-path partition schema, Spark's - * [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] cannot match - * [[org.apache.spark.sql.catalyst.expressions.GetStructField]]-based filter references against it and therefore passes empty - * `partitionFilters` here — which would cause all partitions to be scanned. + * For tables with nested partition columns (e.g. `nested_record.level`), + * [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]] exposes a flat dot-path + * partition schema that Spark's [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] + * cannot match against [[org.apache.spark.sql.catalyst.expressions.GetStructField]]-based filter + * references, so it passes empty `partitionFilters` here. In that case the nested partition + * predicates end up in `dataFilters` instead; we pass them through to [[filterFileSlices]] where + * [[listMatchingPartitionPaths]] knows how to extract and apply them for partition pruning. + * + * This approach is fully stateless — every call recomputes from the provided expressions — + * so it is safe under AQE re-planning, subqueries, and FileIndex instance reuse across queries. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - // Use the cached result from HoodiePruneFileSourcePartitions ONLY when FileSourceScanExec - // passes empty partitionFilters. This is the signature of the nested-partition-column case: - // because HadoopFsRelation exposes a flat dot-path partition schema, FileSourceScanExec - // cannot match GetStructField-based filter references and therefore passes partitionFilters=[] - // here, which would cause all partitions to be scanned without the cache. - // - // For regular partition columns FileSourceScanExec always passes non-empty partitionFilters, - // so we call filterFileSlices fresh regardless of the cache. This is critical for correctness: - // the cache may have been populated by a *different* query's HoodiePruneFileSourcePartitions - // run (e.g. from a .executedPlan.toString() call that triggers planning but not execution, - // leaving hasPushedDownPartitionPredicates=true with stale slices), and using it would return - // wrong rows for a query with different filters. - val usedCache = partitionFilters.isEmpty && cachedPrunedSlices.isDefined - val slices = if (usedCache) { - cachedPrunedSlices.get.flatMap { case (partitionOpt, fileSlices) => - fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) - } + // When partitionFilters is empty and the table has nested partition columns, the nested + // partition predicates were misclassified into dataFilters by FileSourceScanExec (because + // the flat dot-path partition schema cannot match GetStructField expressions). Pass + // dataFilters as the partition filters so that listMatchingPartitionPaths can extract + // and apply the nested partition predicates. + val effectivePartitionFilters = if (partitionFilters.isEmpty && hasNestedPartitionColumns) { + dataFilters } else { - // A non-empty partitionFilters means we are on the regular-partition path and must compute - // fresh results. Discard any stale cache so it cannot be picked up by a later - // nested-partition query on the same file-index instance. - cachedPrunedSlices = None - filterFileSlices(dataFilters, partitionFilters).flatMap( - { case (partitionOpt, fileSlices) => - fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) - } - ) - } - val result = prepareFileSlices(slices) - if (usedCache) { - // Reset so that if this file-index instance is reused for a subsequent query execution - // (possible because HoodieTableMetaClient.equals is content-based, causing Spark's optimizer - // to treat two independently-created file indexes for the same table as equivalent and reuse - // one plan for both), HoodiePruneFileSourcePartitions will re-run and repopulate the cache - // with a fresh read of the table state rather than serving stale pre-mutation data. - hasPushedDownPartitionPredicates = false - cachedPrunedSlices = None + partitionFilters } - result + + val slices = filterFileSlices(dataFilters, effectivePartitionFilters).flatMap( + { case (partitionOpt, fileSlices) => + fileSlices.filter(!_.isEmpty).map(fs => (InternalRow.fromSeq(partitionOpt.get.getValues), fs)) + } + ) + prepareFileSlices(slices) } protected def prepareFileSlices(slices: Seq[(InternalRow, FileSlice)]): Seq[PartitionDirectory] = { @@ -263,10 +245,9 @@ case class HoodieFileIndex(spark: SparkSession, * * @param dataFilters data columns filters * @param partitionFilters partition column filters - * @param partitionPrune for HoodiePruneFileSourcePartitions rule only prune partitions * @return A sequence of pruned partitions and corresponding filtered file slices */ - def filterFileSlices(dataFilters: Seq[Expression], partitionFilters: Seq[Expression], isPartitionPruned: Boolean = false) + def filterFileSlices(dataFilters: Seq[Expression], partitionFilters: Seq[Expression]) : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = { val (isPruned, prunedPartitionsAndFileSlices) = @@ -337,14 +318,6 @@ case class HoodieFileIndex(spark: SparkSession, prunedPartitionsAndFilteredFileSlices } - // Cache the post-data-skipping slices so that the subsequent listFiles call (from - // FileSourceScanExec) can reuse them for tables with nested partition columns. - // The cache must be populated after data skipping so that listFiles does not re-read - // files that were already eliminated by column-stats. - if (isPartitionPruned) { - cachedPrunedSlices = Some(result) - } - result } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index aac93d6202512..686570b6fdbba 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -52,7 +52,7 @@ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruned = true) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala index 7d7240231cd09..3e1878e3c8f8b 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala @@ -60,7 +60,7 @@ case class Spark33HoodiePruneFileSourcePartitions(spark: SparkSession) extends R // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruned = true) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index 079b7b1ec8c84..dc0df67d8bd44 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -52,7 +52,7 @@ case class Spark4HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruned = true) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files From 45e6fb37b9270c7569ee70658bd4dd369e3a0628 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 13:01:11 -0700 Subject: [PATCH 09/20] refactor --- .../scala/org/apache/hudi/HoodieFileIndex.scala | 17 +++++++++++++---- 1 file changed, 13 insertions(+), 4 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index ba1eed8a1fbb5..5490a791ea6d4 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -191,11 +191,20 @@ case class HoodieFileIndex(spark: SparkSession, override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { // When partitionFilters is empty and the table has nested partition columns, the nested // partition predicates were misclassified into dataFilters by FileSourceScanExec (because - // the flat dot-path partition schema cannot match GetStructField expressions). Pass - // dataFilters as the partition filters so that listMatchingPartitionPaths can extract - // and apply the nested partition predicates. + // the flat dot-path partition schema cannot match GetStructField expressions). Extract + // only the data filters that reference nested partition columns and use those for partition + // pruning. We must not pass unrelated data filters as partition filters, because + // prunePartitionsAndGetFileSlices branches on partitionFilters.nonEmpty — passing non-empty + // but partition-irrelevant filters would skip the PARTITION_STATS index pruning path. val effectivePartitionFilters = if (partitionFilters.isEmpty && hasNestedPartitionColumns) { - dataFilters + val partitionColumnNames = getPartitionColumns + dataFilters.filter { expr => + expr.references.nonEmpty && expr.references.map(_.name).forall { ref => + val logicalRef = ref.replaceAll("#\\d+$", "") + partitionColumnNames.exists(partCol => + partCol.startsWith(logicalRef + ".") || spark.sessionState.analyzer.resolver(logicalRef, partCol)) + } + } } else { partitionFilters } From aa9c5010a8e6f5aae896602e295f632c798133cc Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 15:22:48 -0700 Subject: [PATCH 10/20] address comments --- .../org/apache/hudi/HoodieFileIndex.scala | 175 ++++++++++++++++-- .../hudi/SparkHoodieTableFileIndex.scala | 12 +- ...park3HoodiePruneFileSourcePartitions.scala | 7 +- ...park4HoodiePruneFileSourcePartitions.scala | 7 +- 4 files changed, 166 insertions(+), 35 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 5490a791ea6d4..d5782dac8546a 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -37,7 +37,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.HoodieCatalogTable -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression, GetStructField, Literal} import org.apache.spark.sql.execution.datasources.{FileIndex, FileStatusCache, NoopCache, PartitionDirectory} import org.apache.spark.sql.hudi.HoodieSqlCommonUtils import org.apache.spark.sql.internal.SQLConf @@ -177,16 +177,39 @@ case class HoodieFileIndex(spark: SparkSession, /** * Invoked by Spark to fetch list of latest base files per partition. * - * For tables with nested partition columns (e.g. `nested_record.level`), - * [[org.apache.spark.sql.execution.datasources.HadoopFsRelation]] exposes a flat dot-path - * partition schema that Spark's [[org.apache.spark.sql.execution.datasources.FileSourceScanExec]] - * cannot match against [[org.apache.spark.sql.catalyst.expressions.GetStructField]]-based filter - * references, so it passes empty `partitionFilters` here. In that case the nested partition - * predicates end up in `dataFilters` instead; we pass them through to [[filterFileSlices]] where - * [[listMatchingPartitionPaths]] knows how to extract and apply them for partition pruning. + * == Regular (non-nested) partition columns == + * + * For regular partition columns (e.g. `country`), Spark's [[FileSourceScanExec]] correctly + * classifies partition predicates into `partitionFilters` and passes them here. This method + * forwards them directly to [[filterFileSlices]] → [[prunePartitionsAndGetFileSlices]] → + * [[listMatchingPartitionPaths]] for partition pruning. Spark handles the full filter + * classification lifecycle — including splitting top-level AND conjuncts and identifying which + * filters reference only partition columns vs. data columns. Filters like `(a = 1 OR d = 2)` + * that mix partition and data columns are correctly classified as data filters by Spark (since + * `references` is not a subset of partition columns), so they are NOT used for partition + * pruning — which is correct because the `d = 2` branch means any partition could match. + * + * == Nested partition columns == + * + * For nested partition columns (e.g. `nested_record.level`), [[HadoopFsRelation]] exposes a + * flat dot-path partition schema that [[FileSourceScanExec]] cannot match against + * [[GetStructField]]-based filter references, so it passes empty `partitionFilters` here. + * The nested partition predicates end up in `dataFilters` instead. We re-extract them via + * [[extractNestedPartitionFilters]] and pass them through to [[filterFileSlices]]. See that + * method's Scaladoc for known limitations of the re-extraction (mixed AND/OR expressions). * * This approach is fully stateless — every call recomputes from the provided expressions — * so it is safe under AQE re-planning, subqueries, and FileIndex instance reuse across queries. + * + * == Performance == + * + * [[HoodiePruneFileSourcePartitions]] also calls [[filterFileSlices]] during logical + * optimization (for plan statistics), so partition pruning and data skipping may run twice — + * once during planning and once here during execution. This is the same behavior as regular + * (non-nested) partition columns. Partition path listing is cached internally by + * [[SparkHoodieTableFileIndex]], but metadata table lookups for data skipping are not cached + * and will execute twice. A future optimization could cache the data-skipping result keyed + * by the query execution ID to avoid the redundant metadata table lookup. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { // When partitionFilters is empty and the table has nested partition columns, the nested @@ -196,15 +219,16 @@ case class HoodieFileIndex(spark: SparkSession, // pruning. We must not pass unrelated data filters as partition filters, because // prunePartitionsAndGetFileSlices branches on partitionFilters.nonEmpty — passing non-empty // but partition-irrelevant filters would skip the PARTITION_STATS index pruning path. + // + // Known limitation: for tables with BOTH flat and nested partition columns (e.g. + // ["country", "nested_record.level"]), Spark may pass partitionFilters = [country = 'US'] + // (it recognizes the flat column) while the nested partition predicate ends up in + // dataFilters. Since partitionFilters.nonEmpty, we skip extraction and the nested filter + // is not used for partition pruning. This is acceptable because mixed flat+nested partition + // schemas are unusual. A future fix could always attempt extraction when + // hasNestedPartitionColumns is true and merge the results with partitionFilters. val effectivePartitionFilters = if (partitionFilters.isEmpty && hasNestedPartitionColumns) { - val partitionColumnNames = getPartitionColumns - dataFilters.filter { expr => - expr.references.nonEmpty && expr.references.map(_.name).forall { ref => - val logicalRef = ref.replaceAll("#\\d+$", "") - partitionColumnNames.exists(partCol => - partCol.startsWith(logicalRef + ".") || spark.sessionState.analyzer.resolver(logicalRef, partCol)) - } - } + extractNestedPartitionFilters(dataFilters) } else { partitionFilters } @@ -217,6 +241,14 @@ case class HoodieFileIndex(spark: SparkSession, prepareFileSlices(slices) } + /** + * Extracts filters from `dataFilters` that actually reference nested partition columns. + * Delegates to the companion object method with the current table's partition columns. + */ + private def extractNestedPartitionFilters(dataFilters: Seq[Expression]): Seq[Expression] = { + HoodieFileIndex.extractNestedPartitionFilters(dataFilters, getPartitionColumns.toSet) + } + protected def prepareFileSlices(slices: Seq[(InternalRow, FileSlice)]): Seq[PartitionDirectory] = { hasPushedDownPartitionPredicates = true @@ -539,6 +571,117 @@ object HoodieFileIndex extends Logging { val Strict: Val = Val("strict") } + /** + * Extracts filters from `dataFilters` that reference nested partition columns. + * + * == Background == + * + * For regular partition columns, Spark's [[org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions]] + * (or Hudi's [[HoodiePruneFileSourcePartitions]]) classifies each filter expression by checking + * whether `expr.references` is a subset of the partition column [[AttributeSet]]. This works + * because flat partition columns appear as top-level [[AttributeReference]]s that Spark can + * directly match. Spark also splits top-level AND conjuncts into separate filters before + * classifying, so `a = 1 AND d = 2` becomes two filters: `a = 1` (partition) and `d = 2` + * (data). An OR like `(a = 1 OR d = 2)` stays as one expression whose references include + * both `a` and `d`, so it's correctly classified as a data filter (the `d = 2` branch means + * any partition could match). + * + * For nested partition columns (e.g. `nested_record.level`), this classification breaks down. + * [[FileSourceScanExec]] cannot match [[GetStructField]] expressions against the flat dot-path + * partition schema, so all nested partition predicates end up in `dataFilters`. This method + * re-extracts them by walking [[GetStructField]] chains to reconstruct the full dot-path and + * checking whether it matches a partition column name. + * + * We cannot rely solely on [[AttributeReference]] name matching (the struct root) because + * multiple nested fields may share the same root struct (e.g. `nested_record.level` and + * `nested_record.other_field` both reference `nested_record`). + * + * == Behavior compared to regular partition columns == + * + * For simple predicates like `nested_record.level = 'INFO'`, this method behaves identically + * to Spark's classification of regular partition filters — the predicate is extracted and used + * for partition pruning. + * + * == Known limitations (not present for regular partition columns) == + * + * Because this method operates on already-classified `dataFilters` rather than the raw filter + * list, it cannot re-split AND conjuncts that Spark may have bundled into a single expression. + * This leads to two cases where nested partition pruning is less effective than regular: + * + * - '''Mixed AND as single expression''': If `a.b.c = 1 AND d = 2` is passed as a single + * expression (rather than two separate conjuncts), it is excluded entirely because `d` is + * not a partition column root. In practice Spark splits top-level AND conjuncts before + * passing them to [[FileSourceScanExec]], so this mainly affects AND expressions nested + * inside an OR branch. + * + * - '''OR predicates mixing partition and data columns''': `(a.b.c = 1 AND d = 2) OR + * (a.b.c = 3)` is a single expression referencing both partition and data columns, so it + * is excluded. A more sophisticated implementation could extract a weaker partition-only + * predicate (e.g. `a.b.c IN (1, 3)`) to enable partition pruning while still applying the + * full predicate as a post-scan filter. Note that Spark's own + * [[org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions]] has the same + * limitation for regular partition columns with OR predicates — `(a = 1 OR d = 2)` is + * classified as a data filter, not a partition filter. The difference is that for regular + * columns Spark correctly extracts the pure-partition conjuncts from the top-level AND + * before encountering the OR, whereas for nested columns we may miss some of those. + * + * @param dataFilters filters to scan for nested partition predicates + * @param partitionColumnNames the set of partition column dot-paths (e.g. Set("nested_record.level")) + * @return only the filters that exclusively reference partition columns + */ + private[hudi] def extractNestedPartitionFilters(dataFilters: Seq[Expression], + partitionColumnNames: Set[String]): Seq[Expression] = { + val partitionColumnRoots = partitionColumnNames.map(_.split("\\.", 2)(0)) + dataFilters.filter { expr => + // Resolve all outermost GetStructField chains to their full dot-paths. + val structFieldPaths = collectOutermostStructFieldPaths(expr) + // The expression is a partition filter only when: + // 1. It contains at least one GetStructField that resolves to a partition column path, AND + // 2. ALL resolved paths are partition columns (no non-partition nested fields), AND + // 3. ALL attribute references are roots of partition columns + // (guards against mixed expressions like "nested_record.level = 'INFO' AND int_field > 0") + structFieldPaths.nonEmpty && + structFieldPaths.forall(partitionColumnNames.contains) && + expr.references.map(r => stripExprIdSuffix(r.name)).forall(partitionColumnRoots.contains) + } + } + + /** + * Collects the full dot-paths of outermost [[GetStructField]] chains in an expression tree. + * For `EqualTo(GetStructField(GetStructField(attr("a"), _, "b"), _, "c"), Literal(1))`, + * returns `Seq("a.b.c")` — only the outermost chain, not intermediate segments like "a.b". + */ + private[hudi] def collectOutermostStructFieldPaths(expr: Expression): Seq[String] = { + expr match { + case g: GetStructField => + // This is an outermost GetStructField — resolve the full chain and don't recurse + // into children (they are intermediate segments of the same chain). + resolveGetStructFieldPath(g).toSeq + case _ => + // Not a GetStructField — recurse into children to find GetStructField chains. + expr.children.flatMap(collectOutermostStructFieldPaths) + } + } + + /** + * Resolves a [[GetStructField]] chain to its full dot-path string. + * E.g. `GetStructField(GetStructField(attr("a"), _, "b"), _, "c")` resolves to `"a.b.c"`. + */ + private[hudi] def resolveGetStructFieldPath(expr: Expression): Option[String] = expr match { + case GetStructField(child: AttributeReference, _, Some(fieldName)) => + Some(stripExprIdSuffix(child.name) + "." + fieldName) + case GetStructField(child: GetStructField, _, Some(fieldName)) => + resolveGetStructFieldPath(child).map(_ + "." + fieldName) + case _ => None + } + + /** + * Strips Spark's internal exprId suffix (e.g. `#136`) from an attribute name. + * Filter expressions may reference columns with these suffixed names (e.g. `nested_record#136`), + * while partition schema uses logical names (e.g. `nested_record`). + */ + private[hudi] def stripExprIdSuffix(name: String): String = name.replaceAll("#\\d+$", "") + def collectReferencedColumns(spark: SparkSession, queryFilters: Seq[Expression], schema: StructType): Seq[String] = { val resolver = spark.sessionState.analyzer.resolver val refs = queryFilters.flatMap(_.references) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 6fdaade69aa5f..d53303d86db09 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -260,11 +260,9 @@ class SparkHoodieTableFileIndex(spark: SparkSession, def listMatchingPartitionPaths(predicates: Seq[Expression]): Seq[PartitionPath] = { val resolve = spark.sessionState.analyzer.resolver val partitionColumnNames = getPartitionColumns - // Strip Spark's internal exprId suffix (e.g. #136) so nested_record#136 matches nested_record.level - def logicalRefName(ref: String): String = ref.replaceAll("#\\d+$", "") val partitionPruningPredicates = predicates.filter { _.references.map(_.name).forall { ref => - val logicalRef = logicalRefName(ref) + val logicalRef = HoodieFileIndex.stripExprIdSuffix(ref) // NOTE: We're leveraging Spark's resolver here to appropriately handle case-sensitivity. // For nested partition columns (e.g. nested_record.level), ref may be the struct root // (e.g. nested_record#136); match when logicalRef equals partCol or is a prefix of partCol. @@ -300,7 +298,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val partitionFieldNames = partitionSchema.fieldNames def getPartitionColumnPath(expr: Expression): Option[String] = expr match { case a: AttributeReference => - Some(a.name.replaceAll("#\\d+$", "")) + Some(HoodieFileIndex.stripExprIdSuffix(a.name)) case GetStructField(child, _, Some(fieldName)) => getPartitionColumnPath(child).map(_ + "." + fieldName) case _ => None @@ -323,17 +321,17 @@ class SparkHoodieTableFileIndex(spark: SparkSession, case n @ IsNotNull(_) if n.references.map(_.name).nonEmpty && n.references.map(_.name).forall { ref => - val logicalName = ref.replaceAll("#\\d+$", "") + val logicalName = HoodieFileIndex.stripExprIdSuffix(ref) partitionFieldNames.exists(_.startsWith(logicalName + ".")) } => Literal(true) case n @ IsNull(_) if n.references.map(_.name).nonEmpty && n.references.map(_.name).forall { ref => - val logicalName = ref.replaceAll("#\\d+$", "") + val logicalName = HoodieFileIndex.stripExprIdSuffix(ref) partitionFieldNames.exists(_.startsWith(logicalName + ".")) } => Literal(false) case a: AttributeReference => - val logicalName = a.name.replaceAll("#\\d+$", "") + val logicalName = HoodieFileIndex.stripExprIdSuffix(a.name) val index = partitionSchema.indexWhere(sf => resolve(logicalName, sf.name)) if (index >= 0) BoundReference(index, partitionSchema(index).dataType, nullable = true) else a diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index 686570b6fdbba..ab37572f8aa7d 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -105,13 +105,8 @@ private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } - /** - * Returns the logical name of an attribute by stripping Spark's internal exprId suffix (e.g. #136). - * Filter expressions may reference columns with these suffixed names (e.g. nested_record#136.level), - * while partition schema uses logical names (e.g. nested_record.level). - */ private def logicalAttributeName(attr: AttributeReference): String = { - attr.name.replaceAll("#\\d+$", "") + HoodieFileIndex.stripExprIdSuffix(attr.name) } /** diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index dc0df67d8bd44..2316aaa39b65c 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -105,13 +105,8 @@ private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } - /** - * Returns the logical name of an attribute by stripping Spark's internal exprId suffix (e.g. #136). - * Filter expressions may reference columns with these suffixed names (e.g. nested_record#136.level), - * while partition schema uses logical names (e.g. nested_record.level). - */ private def logicalAttributeName(attr: AttributeReference): String = { - attr.name.replaceAll("#\\d+$", "") + HoodieFileIndex.stripExprIdSuffix(attr.name) } /** From c44767c53bb5a4b8ff961ad770a9764250970435 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 15:33:33 -0700 Subject: [PATCH 11/20] Address comments --- .../org/apache/hudi/HoodieFileIndex.scala | 21 ++++++++++++------- ...park3HoodiePruneFileSourcePartitions.scala | 2 +- ...ark33HoodiePruneFileSourcePartitions.scala | 2 +- ...park4HoodiePruneFileSourcePartitions.scala | 2 +- 4 files changed, 16 insertions(+), 11 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index d5782dac8546a..fe6e4ea9ac88e 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -204,12 +204,10 @@ case class HoodieFileIndex(spark: SparkSession, * == Performance == * * [[HoodiePruneFileSourcePartitions]] also calls [[filterFileSlices]] during logical - * optimization (for plan statistics), so partition pruning and data skipping may run twice — - * once during planning and once here during execution. This is the same behavior as regular - * (non-nested) partition columns. Partition path listing is cached internally by - * [[SparkHoodieTableFileIndex]], but metadata table lookups for data skipping are not cached - * and will execute twice. A future optimization could cache the data-skipping result keyed - * by the query execution ID to avoid the redundant metadata table lookup. + * optimization, but with `isPartitionPruneOnly = true` which skips the expensive metadata + * table lookup for data skipping. Data skipping runs only once — here during execution. + * Partition path listing may run twice (planning + execution) but is cached internally by + * [[SparkHoodieTableFileIndex]]. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { // When partitionFilters is empty and the table has nested partition columns, the nested @@ -286,9 +284,13 @@ case class HoodieFileIndex(spark: SparkSession, * * @param dataFilters data columns filters * @param partitionFilters partition column filters + * @param isPartitionPruneOnly when true, skip data skipping (metadata table lookup) and only prune partitions. + * Set to true by [[HoodiePruneFileSourcePartitions]] which only needs partition + * pruning for plan statistics — data skipping will run later in [[listFiles]]. * @return A sequence of pruned partitions and corresponding filtered file slices */ - def filterFileSlices(dataFilters: Seq[Expression], partitionFilters: Seq[Expression]) + def filterFileSlices(dataFilters: Seq[Expression], partitionFilters: Seq[Expression], + isPartitionPruneOnly: Boolean = false) : Seq[(Option[BaseHoodieTableFileIndex.PartitionPath], Seq[FileSlice])] = { val (isPruned, prunedPartitionsAndFileSlices) = @@ -296,8 +298,11 @@ case class HoodieFileIndex(spark: SparkSession, hasPushedDownPartitionPredicates = true // If there are no data filters, return all the file slices. + // If isPartitionPruneOnly is true, this is called from HoodiePruneFileSourcePartitions for plan + // statistics only — skip the expensive metadata table lookup for data skipping, which will run + // later when listFiles calls filterFileSlices during execution. // If there are no file slices, return empty list. - val result = if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty) { + val result = if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty || isPartitionPruneOnly) { prunedPartitionsAndFileSlices } else { // Look up candidate files names in the col-stats or record level index, if all of the following conditions are true diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index ab37572f8aa7d..66766baec6cf9 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -52,7 +52,7 @@ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruneOnly = true) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files diff --git a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala index 3e1878e3c8f8b..add1b7aaebf14 100644 --- a/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3.3.x/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark33HoodiePruneFileSourcePartitions.scala @@ -60,7 +60,7 @@ case class Spark33HoodiePruneFileSourcePartitions(spark: SparkSession) extends R // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruneOnly = true) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index 2316aaa39b65c..5983096c3b57a 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -52,7 +52,7 @@ case class Spark4HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru // [[HudiFileIndex]] is a caching one, therefore we don't need to reconstruct new relation, // instead we simply just refresh the index and update the stats - fileIndex.filterFileSlices(dataFilters, partitionPruningFilters) + fileIndex.filterFileSlices(dataFilters, partitionPruningFilters, isPartitionPruneOnly = true) if (partitionPruningFilters.nonEmpty) { // Change table stats based on the sizeInBytes of pruned files From 0b85f2e920fb1adc9427c75cf28f436ca82b9743 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 15:57:14 -0700 Subject: [PATCH 12/20] address comments --- .../org/apache/hudi/HoodieFileIndex.scala | 11 ++----- .../hudi/SparkHoodieTableFileIndex.scala | 32 ++++++++++--------- ...park3HoodiePruneFileSourcePartitions.scala | 13 +++----- ...park4HoodiePruneFileSourcePartitions.scala | 13 +++----- 4 files changed, 27 insertions(+), 42 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index fe6e4ea9ac88e..73cf917cd36dc 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -647,7 +647,7 @@ object HoodieFileIndex extends Logging { // (guards against mixed expressions like "nested_record.level = 'INFO' AND int_field > 0") structFieldPaths.nonEmpty && structFieldPaths.forall(partitionColumnNames.contains) && - expr.references.map(r => stripExprIdSuffix(r.name)).forall(partitionColumnRoots.contains) + expr.references.map(_.name).forall(partitionColumnRoots.contains) } } @@ -674,19 +674,12 @@ object HoodieFileIndex extends Logging { */ private[hudi] def resolveGetStructFieldPath(expr: Expression): Option[String] = expr match { case GetStructField(child: AttributeReference, _, Some(fieldName)) => - Some(stripExprIdSuffix(child.name) + "." + fieldName) + Some(child.name + "." + fieldName) case GetStructField(child: GetStructField, _, Some(fieldName)) => resolveGetStructFieldPath(child).map(_ + "." + fieldName) case _ => None } - /** - * Strips Spark's internal exprId suffix (e.g. `#136`) from an attribute name. - * Filter expressions may reference columns with these suffixed names (e.g. `nested_record#136`), - * while partition schema uses logical names (e.g. `nested_record`). - */ - private[hudi] def stripExprIdSuffix(name: String): String = name.replaceAll("#\\d+$", "") - def collectReferencedColumns(spark: SparkSession, queryFilters: Seq[Expression], schema: StructType): Seq[String] = { val resolver = spark.sessionState.analyzer.resolver val refs = queryFilters.flatMap(_.references) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index d53303d86db09..8787f183f03aa 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -150,11 +150,8 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val partitionFields: Array[StructField] = partitionColumns.get().map(column => StructField(column, StringType)) StructType(partitionFields) } else { - // Use full partition path (e.g. "nested_record.level") as the partition column name so that - // data schema does not exclude a same-named top-level column (e.g. "level") when partition - // path is a nested field. Otherwise partition value would overwrite the data column on read. val partitionFields: Array[StructField] = partitionColumns.get().filter(column => nameFieldMap.contains(column)) - .map(column => StructField(column, nameFieldMap.apply(column).dataType)) + .map(column => nameFieldMap.apply(column)) if (partitionFields.length != partitionColumns.get().length) { val isBootstrapTable = tableConfig.getBootstrapBasePath.isPresent @@ -262,12 +259,11 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val partitionColumnNames = getPartitionColumns val partitionPruningPredicates = predicates.filter { _.references.map(_.name).forall { ref => - val logicalRef = HoodieFileIndex.stripExprIdSuffix(ref) // NOTE: We're leveraging Spark's resolver here to appropriately handle case-sensitivity. // For nested partition columns (e.g. nested_record.level), ref may be the struct root - // (e.g. nested_record#136); match when logicalRef equals partCol or is a prefix of partCol. + // (e.g. nested_record); match when ref equals partCol or is a prefix of partCol. partitionColumnNames.exists(partCol => - resolve(logicalRef, partCol) || partCol.startsWith(logicalRef + ".")) + resolve(ref, partCol) || partCol.startsWith(ref + ".")) } } @@ -298,7 +294,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val partitionFieldNames = partitionSchema.fieldNames def getPartitionColumnPath(expr: Expression): Option[String] = expr match { case a: AttributeReference => - Some(HoodieFileIndex.stripExprIdSuffix(a.name)) + Some(a.name) case GetStructField(child, _, Some(fieldName)) => getPartitionColumnPath(child).map(_ + "." + fieldName) case _ => None @@ -321,18 +317,15 @@ class SparkHoodieTableFileIndex(spark: SparkSession, case n @ IsNotNull(_) if n.references.map(_.name).nonEmpty && n.references.map(_.name).forall { ref => - val logicalName = HoodieFileIndex.stripExprIdSuffix(ref) - partitionFieldNames.exists(_.startsWith(logicalName + ".")) + partitionFieldNames.exists(_.startsWith(ref + ".")) } => Literal(true) case n @ IsNull(_) if n.references.map(_.name).nonEmpty && n.references.map(_.name).forall { ref => - val logicalName = HoodieFileIndex.stripExprIdSuffix(ref) - partitionFieldNames.exists(_.startsWith(logicalName + ".")) + partitionFieldNames.exists(_.startsWith(ref + ".")) } => Literal(false) case a: AttributeReference => - val logicalName = HoodieFileIndex.stripExprIdSuffix(a.name) - val index = partitionSchema.indexWhere(sf => resolve(logicalName, sf.name)) + val index = partitionSchema.indexWhere(sf => resolve(a.name, sf.name)) if (index >= 0) BoundReference(index, partitionSchema(index).dataType, nullable = true) else a } @@ -674,13 +667,22 @@ object SparkHoodieTableFileIndex extends SparkAdapterSupport { * "a.c" -> StructField("c", IntType), * */ + /** + * Builds a map from dot-path field names to [[StructField]]s for all leaf fields in a schema. + * For nested structs, both the key and the [[StructField.name]] use the full dot-path. + * E.g. for schema `StructType(StructField("a", StructType(StructField("b", IntegerType))))`, + * returns `Map("a.b" -> StructField("a.b", IntegerType))`. + */ private def generateFieldMap(structType: StructType) : Map[String, StructField] = { def traverse(structField: Either[StructField, StructType]) : Map[String, StructField] = { structField match { case Right(struct) => struct.fields.flatMap(f => traverse(Left(f))).toMap case Left(field) => field.dataType match { case struct: StructType => traverse(Right(struct)).map { - case (key, structField) => (s"${field.name}.$key", structField) + case (key, structField) => { + val fullPath = s"${field.name}.$key" + (fullPath, structField.copy(name = fullPath)) + } } case _ => Map(field.name -> field) } diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index 66766baec6cf9..1595d949c9aad 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -105,19 +105,14 @@ private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } - private def logicalAttributeName(attr: AttributeReference): String = { - HoodieFileIndex.stripExprIdSuffix(attr.name) - } - /** * Returns true if the given attribute references a partition column. An attribute references a - * partition column if its logical name (without #exprId) equals a partition column name or - * is the struct parent of a nested partition path (e.g. nested_record for nested_record.level). + * partition column if its name equals a partition column name or is the struct parent of a + * nested partition path (e.g. nested_record for nested_record.level). */ private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { - val logicalName = logicalAttributeName(attr) - partitionSchema.names.contains(logicalName) || - partitionSchema.names.exists(_.startsWith(logicalName + ".")) + partitionSchema.names.contains(attr.name) || + partitionSchema.names.exists(_.startsWith(attr.name + ".")) } def getPartitionFiltersAndDataFilters(partitionSchema: StructType, diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index 5983096c3b57a..bd7543572c832 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -105,19 +105,14 @@ private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { Project(projects, withFilter) } - private def logicalAttributeName(attr: AttributeReference): String = { - HoodieFileIndex.stripExprIdSuffix(attr.name) - } - /** * Returns true if the given attribute references a partition column. An attribute references a - * partition column if its logical name (without #exprId) equals a partition column name or - * is the struct parent of a nested partition path (e.g. nested_record for nested_record.level). + * partition column if its name equals a partition column name or is the struct parent of a + * nested partition path (e.g. nested_record for nested_record.level). */ private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { - val logicalName = logicalAttributeName(attr) - partitionSchema.names.contains(logicalName) || - partitionSchema.names.exists(_.startsWith(logicalName + ".")) + partitionSchema.names.contains(attr.name) || + partitionSchema.names.exists(_.startsWith(attr.name + ".")) } def getPartitionFiltersAndDataFilters(partitionSchema: StructType, From 2d85550c77ed93df84e7f437eb13c17d74924046 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 16:36:32 -0700 Subject: [PATCH 13/20] refactor --- .../hudi/SparkHoodieTableFileIndex.scala | 66 ++++---- ...park3HoodiePruneFileSourcePartitions.scala | 12 +- ...estGetPartitionFiltersAndDataFilters.scala | 149 ++++++++++++++++++ ...park4HoodiePruneFileSourcePartitions.scala | 12 +- 4 files changed, 191 insertions(+), 48 deletions(-) create mode 100644 hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 8787f183f03aa..80c78accf5454 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -46,7 +46,7 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, IsNotNull, IsNull, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf @@ -257,16 +257,35 @@ class SparkHoodieTableFileIndex(spark: SparkSession, def listMatchingPartitionPaths(predicates: Seq[Expression]): Seq[PartitionPath] = { val resolve = spark.sessionState.analyzer.resolver val partitionColumnNames = getPartitionColumns - val partitionPruningPredicates = predicates.filter { - _.references.map(_.name).forall { ref => - // NOTE: We're leveraging Spark's resolver here to appropriately handle case-sensitivity. - // For nested partition columns (e.g. nested_record.level), ref may be the struct root - // (e.g. nested_record); match when ref equals partCol or is a prefix of partCol. - partitionColumnNames.exists(partCol => - resolve(ref, partCol) || partCol.startsWith(ref + ".")) - } + + // Resolve a GetStructField chain to its full dot-path (e.g. "nested_record.level"). + def getFieldPath(expr: Expression): Option[String] = expr match { + case a: AttributeReference => Some(a.name) + case GetStructField(child, _, Some(fieldName)) => + getFieldPath(child).map(_ + "." + fieldName) + case _ => None + } + + // Returns true if every column reference in the expression resolves to a partition column. + // For flat columns (e.g. "country"), checks AttributeReference.name directly. + // For nested columns (e.g. "nested_record.level"), walks GetStructField chains to build + // the full dot-path and checks whether it matches a partition column name. + // This avoids the overly broad struct-parent prefix matching that would misclassify + // filters on non-partition nested fields (e.g. "nested_record.nested_int") as partition filters. + def referencesOnlyPartitionColumns(expr: Expression): Boolean = expr match { + case g: GetStructField => + getFieldPath(g).exists(path => partitionColumnNames.exists(pc => resolve(path, pc))) + case _: AttributeReference => + // Flat attribute — check if it's a partition column directly + partitionColumnNames.exists(pc => resolve(expr.asInstanceOf[AttributeReference].name, pc)) + case _ => + // For compound expressions (And, Or, EqualTo, etc.), all children must reference + // only partition columns. Literals have no children and pass vacuously. + expr.children.forall(referencesOnlyPartitionColumns) } + val partitionPruningPredicates = predicates.filter(referencesOnlyPartitionColumns) + if (partitionPruningPredicates.isEmpty) { val queryPartitionPaths = getAllQueryPartitionPaths.asScala.toSeq logInfo(s"No partition predicates provided, listing full table (${queryPartitionPaths.size} partitions)") @@ -292,38 +311,13 @@ class SparkHoodieTableFileIndex(spark: SparkSession, if (haveProperPartitionValues(partitionPaths.toSeq) && partitionSchema.nonEmpty) { val predicate = partitionPruningPredicates.reduce(expressions.And) val partitionFieldNames = partitionSchema.fieldNames - def getPartitionColumnPath(expr: Expression): Option[String] = expr match { - case a: AttributeReference => - Some(a.name) - case GetStructField(child, _, Some(fieldName)) => - getPartitionColumnPath(child).map(_ + "." + fieldName) - case _ => None - } val transformedPredicate = predicate.transform { case g @ GetStructField(_, _, Some(_)) => - getPartitionColumnPath(g).flatMap { path => - val idx = partitionFieldNames.indexOf(path) + getFieldPath(g).flatMap { path => + val idx = partitionFieldNames.indexWhere(name => resolve(path, name)) if (idx >= 0) Some(BoundReference(idx, partitionSchema(idx).dataType, nullable = true)) else None }.getOrElse(g) - // NOTE: Spark's optimizer auto-adds IsNotNull(struct_attr) when a nested field is - // filtered (e.g. IsNotNull(nested_record) for filter nested_record.level = 'INFO'). - // The struct attribute cannot be directly bound to a flat partition value. - // Since partition rows are parsed from partition paths they are never null, so - // IsNotNull for any expression whose references are all struct-parents of nested - // partition columns is always true (and IsNull always false). - // We match on the expression's references rather than requiring the child to be a - // plain AttributeReference, because the optimizer may wrap it in casts or other nodes. - case n @ IsNotNull(_) - if n.references.map(_.name).nonEmpty && - n.references.map(_.name).forall { ref => - partitionFieldNames.exists(_.startsWith(ref + ".")) - } => Literal(true) - case n @ IsNull(_) - if n.references.map(_.name).nonEmpty && - n.references.map(_.name).forall { ref => - partitionFieldNames.exists(_.startsWith(ref + ".")) - } => Literal(false) case a: AttributeReference => val index = partitionSchema.indexWhere(sf => resolve(a.name, sf.name)) if (index >= 0) BoundReference(index, partitionSchema(index).dataType, nullable = true) diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index 1595d949c9aad..b41b367514b51 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -81,7 +81,7 @@ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru } -private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { +private[analysis] object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { private val exprUtils = sparkAdapter.getCatalystExpressionUtils @@ -106,13 +106,13 @@ private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { } /** - * Returns true if the given attribute references a partition column. An attribute references a - * partition column if its name equals a partition column name or is the struct parent of a - * nested partition path (e.g. nested_record for nested_record.level). + * Returns true if the given attribute references a partition column. For nested partition columns + * (e.g. `nested_record.level`), `partitionSchema` is the nested [[StructType]] from + * `partitionSchemaForSpark`, so the top-level name is the struct root (e.g. `nested_record`), + * which matches `attr.name` directly via `contains`. */ private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { - partitionSchema.names.contains(attr.name) || - partitionSchema.names.exists(_.startsWith(attr.name + ".")) + partitionSchema.names.contains(attr.name) } def getPartitionFiltersAndDataFilters(partitionSchema: StructType, diff --git a/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala b/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala new file mode 100644 index 0000000000000..17bac4cde5287 --- /dev/null +++ b/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala @@ -0,0 +1,149 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.spark.sql.hudi.analysis + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ +import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Test + +/** + * Unit tests for [[Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters]]. + * + * Tests that partition filter classification correctly handles nested partition columns + * using the nested [[StructType]] partition schema from `partitionSchemaForSpark`. + */ +class TestGetPartitionFiltersAndDataFilters { + + private def attr(name: String, dataType: DataType = StringType): AttributeReference = + AttributeReference(name, dataType)() + + // Nested partition schema for partition column "nested_record.level" + // partitionSchemaForSpark produces: StructType(StructField("nested_record", StructType(StructField("level", StringType)))) + private val nestedPartitionSchema = StructType(Seq( + StructField("nested_record", StructType(Seq(StructField("level", StringType)))) + )) + + // Flat partition schema for partition column "country" + private val flatPartitionSchema = StructType(Seq( + StructField("country", StringType) + )) + + @Test + def testFlatPartitionColumnRecognized(): Unit = { + // country = 'US' + val filter = EqualTo(attr("country"), Literal("US")) + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(flatPartitionSchema, Seq(filter)) + assertEquals(1, partFilters.size, "Flat partition filter should be classified as partition filter") + assertTrue(dataFilters.isEmpty, "No data filters expected") + } + + @Test + def testFlatDataColumnExcluded(): Unit = { + // int_field = 5 + val filter = EqualTo(attr("int_field", IntegerType), Literal(5)) + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(flatPartitionSchema, Seq(filter)) + assertTrue(partFilters.isEmpty, "Data column filter should not be a partition filter") + assertEquals(1, dataFilters.size) + } + + @Test + def testNestedPartitionColumnStructRootRecognized(): Unit = { + // For nested partition column "nested_record.level", the nested schema has "nested_record" + // as a top-level field. Spark represents filter nested_record.level = 'INFO' as + // GetStructField(attr("nested_record"), ..., "level") = Literal("INFO"). + // The AttributeReference "nested_record" should be recognized as referencing a partition column. + val nestedRecordType = StructType(Seq(StructField("level", StringType))) + val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("level")) + val filter = EqualTo(gsf, Literal("INFO")) + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) + assertEquals(1, partFilters.size, "Nested partition filter should be classified as partition filter") + } + + @Test + def testNonPartitionNestedFieldExcluded(): Unit = { + // nested_record.nested_int = 10 — same struct root but different field + // In the nested schema, "nested_record" is a top-level name, so the AttributeReference + // "nested_record" still matches. This means the filter IS classified as a partition filter + // by getPartitionFiltersAndDataFilters (which uses AttributeSet-based classification). + // The precise filtering happens downstream in listMatchingPartitionPaths via + // referencesOnlyPartitionColumns, which walks GetStructField chains. + val nestedRecordType = StructType(Seq(StructField("nested_int", IntegerType), StructField("level", StringType))) + val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("nested_int")) + val filter = EqualTo(gsf, Literal(10)) + val (partFilters, _) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) + // Note: getPartitionFiltersAndDataFilters classifies by AttributeReference root ("nested_record"), + // which IS in the nested partition schema. So this is classified as a partition filter here. + // The downstream referencesOnlyPartitionColumns in listMatchingPartitionPaths will correctly + // exclude it by checking the full GetStructField path "nested_record.nested_int". + assertEquals(1, partFilters.size, + "Filter on non-partition nested field is classified as partition filter by AttributeSet matching " + + "(precise exclusion happens downstream in listMatchingPartitionPaths)") + } + + @Test + def testMixedPartitionAndDataColumns(): Unit = { + // Two filters: nested_record.level = 'INFO' and int_field = 5 + val nestedRecordType = StructType(Seq(StructField("level", StringType))) + val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("level")) + val partFilter = EqualTo(gsf, Literal("INFO")) + val dataFilter = EqualTo(attr("int_field", IntegerType), Literal(5)) + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(partFilter, dataFilter)) + assertEquals(1, partFilters.size, "Partition filter should be extracted") + assertEquals(1, dataFilters.size, "Data filter should remain") + } + + @Test + def testIsNotNullOnStructRootClassifiedAsPartition(): Unit = { + // IsNotNull(nested_record) — Spark auto-adds this. + // Since "nested_record" is in the nested partition schema names, it's classified as partition. + val nestedRecordType = StructType(Seq(StructField("level", StringType))) + val filter = IsNotNull(attr("nested_record", nestedRecordType)) + val (partFilters, _) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) + assertEquals(1, partFilters.size, + "IsNotNull on struct root should be classified as partition filter (struct root is in nested schema)") + } + + @Test + def testUnrelatedStructNotRecognized(): Unit = { + // other_struct.field = 'value' — struct root "other_struct" is NOT in partition schema + val otherType = StructType(Seq(StructField("field", StringType))) + val gsf = GetStructField(attr("other_struct", otherType), 0, Some("field")) + val filter = EqualTo(gsf, Literal("value")) + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) + assertTrue(partFilters.isEmpty, "Filter on unrelated struct should not be a partition filter") + assertEquals(1, dataFilters.size) + } + + @Test + def testEmptyFilters(): Unit = { + val (partFilters, dataFilters) = + Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq.empty) + assertTrue(partFilters.isEmpty) + assertTrue(dataFilters.isEmpty) + } +} diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index bd7543572c832..faad50e8706fc 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -81,7 +81,7 @@ case class Spark4HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru } -private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { +private[analysis] object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { private val exprUtils = sparkAdapter.getCatalystExpressionUtils @@ -106,13 +106,13 @@ private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { } /** - * Returns true if the given attribute references a partition column. An attribute references a - * partition column if its name equals a partition column name or is the struct parent of a - * nested partition path (e.g. nested_record for nested_record.level). + * Returns true if the given attribute references a partition column. For nested partition columns + * (e.g. `nested_record.level`), `partitionSchema` is the nested [[StructType]] from + * `partitionSchemaForSpark`, so the top-level name is the struct root (e.g. `nested_record`), + * which matches `attr.name` directly via `contains`. */ private def isPartitionColumnReference(attr: AttributeReference, partitionSchema: StructType): Boolean = { - partitionSchema.names.contains(attr.name) || - partitionSchema.names.exists(_.startsWith(attr.name + ".")) + partitionSchema.names.contains(attr.name) } def getPartitionFiltersAndDataFilters(partitionSchema: StructType, From f8a0bb8d21a9a03a64131b109f13853ec236c407 Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 17:03:06 -0700 Subject: [PATCH 14/20] clean comments --- .../org/apache/hudi/HoodieFileIndex.scala | 180 +++++------------- .../hudi/SparkHoodieTableFileIndex.scala | 52 ++--- .../org/apache/hudi/TestHoodieFileIndex.scala | 111 ++++++++++- 3 files changed, 167 insertions(+), 176 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala index 73cf917cd36dc..2187e5ab6903f 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/HoodieFileIndex.scala @@ -105,13 +105,7 @@ case class HoodieFileIndex(spark: SparkSession, @transient protected var hasPushedDownPartitionPredicates: Boolean = false - /** - * True when any partition column name contains a dot, indicating a nested field path - * (e.g. "nested_record.level"). For such columns, Spark's [[FileSourceScanExec]] cannot - * match [[GetStructField]]-based filter references against the flat dot-path partition schema - * exposed by [[HadoopFsRelation]], and therefore passes empty `partitionFilters` to - * [[listFiles]]. We detect this and re-extract partition predicates from `dataFilters`. - */ + /** True when any partition column is a nested field path (e.g. "nested_record.level"). */ private val hasNestedPartitionColumns: Boolean = getPartitionColumns.exists(_.contains(".")) @@ -177,54 +171,25 @@ case class HoodieFileIndex(spark: SparkSession, /** * Invoked by Spark to fetch list of latest base files per partition. * - * == Regular (non-nested) partition columns == - * - * For regular partition columns (e.g. `country`), Spark's [[FileSourceScanExec]] correctly - * classifies partition predicates into `partitionFilters` and passes them here. This method - * forwards them directly to [[filterFileSlices]] → [[prunePartitionsAndGetFileSlices]] → - * [[listMatchingPartitionPaths]] for partition pruning. Spark handles the full filter - * classification lifecycle — including splitting top-level AND conjuncts and identifying which - * filters reference only partition columns vs. data columns. Filters like `(a = 1 OR d = 2)` - * that mix partition and data columns are correctly classified as data filters by Spark (since - * `references` is not a subset of partition columns), so they are NOT used for partition - * pruning — which is correct because the `d = 2` branch means any partition could match. + * For regular partition columns, Spark passes correct `partitionFilters` directly. * - * == Nested partition columns == + * For nested partition columns (e.g. `nested_record.level`), Spark cannot match + * [[GetStructField]] expressions against the flat dot-path partition schema and passes + * `partitionFilters = []`. The nested predicates land in `dataFilters` instead. + * We re-extract them via [[extractNestedPartitionFilters]]. * - * For nested partition columns (e.g. `nested_record.level`), [[HadoopFsRelation]] exposes a - * flat dot-path partition schema that [[FileSourceScanExec]] cannot match against - * [[GetStructField]]-based filter references, so it passes empty `partitionFilters` here. - * The nested partition predicates end up in `dataFilters` instead. We re-extract them via - * [[extractNestedPartitionFilters]] and pass them through to [[filterFileSlices]]. See that - * method's Scaladoc for known limitations of the re-extraction (mixed AND/OR expressions). + * Example: `SELECT * FROM t WHERE nested_record.level = 'INFO' AND int_field > 0` + * - Spark passes: `partitionFilters = []`, `dataFilters = [nested_record.level = 'INFO', int_field > 0]` + * - We extract: `effectivePartitionFilters = [nested_record.level = 'INFO']` * - * This approach is fully stateless — every call recomputes from the provided expressions — - * so it is safe under AQE re-planning, subqueries, and FileIndex instance reuse across queries. + * This is stateless — safe under AQE re-planning, subqueries, and FileIndex reuse. * - * == Performance == - * - * [[HoodiePruneFileSourcePartitions]] also calls [[filterFileSlices]] during logical - * optimization, but with `isPartitionPruneOnly = true` which skips the expensive metadata - * table lookup for data skipping. Data skipping runs only once — here during execution. - * Partition path listing may run twice (planning + execution) but is cached internally by - * [[SparkHoodieTableFileIndex]]. + * Known limitation: for mixed flat+nested partitions (e.g. `["country", "nested_record.level"]`), + * if Spark passes `partitionFilters = [country = 'US']`, we skip extraction and the nested + * filter is not used for partition pruning. A future fix could merge extracted nested filters + * with the provided `partitionFilters`. */ override def listFiles(partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { - // When partitionFilters is empty and the table has nested partition columns, the nested - // partition predicates were misclassified into dataFilters by FileSourceScanExec (because - // the flat dot-path partition schema cannot match GetStructField expressions). Extract - // only the data filters that reference nested partition columns and use those for partition - // pruning. We must not pass unrelated data filters as partition filters, because - // prunePartitionsAndGetFileSlices branches on partitionFilters.nonEmpty — passing non-empty - // but partition-irrelevant filters would skip the PARTITION_STATS index pruning path. - // - // Known limitation: for tables with BOTH flat and nested partition columns (e.g. - // ["country", "nested_record.level"]), Spark may pass partitionFilters = [country = 'US'] - // (it recognizes the flat column) while the nested partition predicate ends up in - // dataFilters. Since partitionFilters.nonEmpty, we skip extraction and the nested filter - // is not used for partition pruning. This is acceptable because mixed flat+nested partition - // schemas are unusual. A future fix could always attempt extraction when - // hasNestedPartitionColumns is true and merge the results with partitionFilters. val effectivePartitionFilters = if (partitionFilters.isEmpty && hasNestedPartitionColumns) { extractNestedPartitionFilters(dataFilters) } else { @@ -239,10 +204,7 @@ case class HoodieFileIndex(spark: SparkSession, prepareFileSlices(slices) } - /** - * Extracts filters from `dataFilters` that actually reference nested partition columns. - * Delegates to the companion object method with the current table's partition columns. - */ + /** Delegates to companion object with this table's partition columns. */ private def extractNestedPartitionFilters(dataFilters: Seq[Expression]): Seq[Expression] = { HoodieFileIndex.extractNestedPartitionFilters(dataFilters, getPartitionColumns.toSet) } @@ -279,15 +241,13 @@ case class HoodieFileIndex(spark: SparkSession, } /** - * The functions prunes the partition paths based on the input partition filters. For every partition path, the file - * slices are further filtered after querying metadata table based on the data filters. + * Prunes partitions by `partitionFilters`, then optionally applies data skipping via metadata + * table indices (column stats, record-level index, etc.) to filter file slices. * - * @param dataFilters data columns filters - * @param partitionFilters partition column filters - * @param isPartitionPruneOnly when true, skip data skipping (metadata table lookup) and only prune partitions. - * Set to true by [[HoodiePruneFileSourcePartitions]] which only needs partition - * pruning for plan statistics — data skipping will run later in [[listFiles]]. - * @return A sequence of pruned partitions and corresponding filtered file slices + * @param dataFilters data column filters (used for data skipping) + * @param partitionFilters partition column filters (used for partition pruning) + * @param isPartitionPruneOnly when true, skip data skipping. Used by [[HoodiePruneFileSourcePartitions]] + * during planning (data skipping runs later in [[listFiles]]). */ def filterFileSlices(dataFilters: Seq[Expression], partitionFilters: Seq[Expression], isPartitionPruneOnly: Boolean = false) @@ -297,12 +257,9 @@ case class HoodieFileIndex(spark: SparkSession, prunePartitionsAndGetFileSlices(dataFilters, partitionFilters) hasPushedDownPartitionPredicates = true - // If there are no data filters, return all the file slices. - // If isPartitionPruneOnly is true, this is called from HoodiePruneFileSourcePartitions for plan - // statistics only — skip the expensive metadata table lookup for data skipping, which will run - // later when listFiles calls filterFileSlices during execution. - // If there are no file slices, return empty list. - val result = if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty || isPartitionPruneOnly) { + // Skip data skipping when: no file slices, no data filters, or partition-prune-only mode + // (planning phase — data skipping runs later during execution). + if (prunedPartitionsAndFileSlices.isEmpty || dataFilters.isEmpty || isPartitionPruneOnly) { prunedPartitionsAndFileSlices } else { // Look up candidate files names in the col-stats or record level index, if all of the following conditions are true @@ -363,8 +320,6 @@ case class HoodieFileIndex(spark: SparkSession, prunedPartitionsAndFilteredFileSlices } - - result } /** @@ -577,62 +532,26 @@ object HoodieFileIndex extends Logging { } /** - * Extracts filters from `dataFilters` that reference nested partition columns. - * - * == Background == - * - * For regular partition columns, Spark's [[org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions]] - * (or Hudi's [[HoodiePruneFileSourcePartitions]]) classifies each filter expression by checking - * whether `expr.references` is a subset of the partition column [[AttributeSet]]. This works - * because flat partition columns appear as top-level [[AttributeReference]]s that Spark can - * directly match. Spark also splits top-level AND conjuncts into separate filters before - * classifying, so `a = 1 AND d = 2` becomes two filters: `a = 1` (partition) and `d = 2` - * (data). An OR like `(a = 1 OR d = 2)` stays as one expression whose references include - * both `a` and `d`, so it's correctly classified as a data filter (the `d = 2` branch means - * any partition could match). + * Extracts filters from `dataFilters` that reference nested partition columns by walking + * [[GetStructField]] chains to reconstruct the full dot-path and matching against partition + * column names. We cannot match on the struct root alone because sibling fields share it + * (e.g. `nested_record.level` and `nested_record.nested_int` both reference `nested_record`). * - * For nested partition columns (e.g. `nested_record.level`), this classification breaks down. - * [[FileSourceScanExec]] cannot match [[GetStructField]] expressions against the flat dot-path - * partition schema, so all nested partition predicates end up in `dataFilters`. This method - * re-extracts them by walking [[GetStructField]] chains to reconstruct the full dot-path and - * checking whether it matches a partition column name. + * Given partition column `nested_record.level` and: + * {{{ + * dataFilters = [nested_record.level = 'INFO', nested_record.nested_int > 0, int_field = 5] + * }}} + * Returns: `[nested_record.level = 'INFO']` * - * We cannot rely solely on [[AttributeReference]] name matching (the struct root) because - * multiple nested fields may share the same root struct (e.g. `nested_record.level` and - * `nested_record.other_field` both reference `nested_record`). - * - * == Behavior compared to regular partition columns == - * - * For simple predicates like `nested_record.level = 'INFO'`, this method behaves identically - * to Spark's classification of regular partition filters — the predicate is extracted and used - * for partition pruning. - * - * == Known limitations (not present for regular partition columns) == - * - * Because this method operates on already-classified `dataFilters` rather than the raw filter - * list, it cannot re-split AND conjuncts that Spark may have bundled into a single expression. - * This leads to two cases where nested partition pruning is less effective than regular: - * - * - '''Mixed AND as single expression''': If `a.b.c = 1 AND d = 2` is passed as a single - * expression (rather than two separate conjuncts), it is excluded entirely because `d` is - * not a partition column root. In practice Spark splits top-level AND conjuncts before - * passing them to [[FileSourceScanExec]], so this mainly affects AND expressions nested - * inside an OR branch. - * - * - '''OR predicates mixing partition and data columns''': `(a.b.c = 1 AND d = 2) OR - * (a.b.c = 3)` is a single expression referencing both partition and data columns, so it - * is excluded. A more sophisticated implementation could extract a weaker partition-only - * predicate (e.g. `a.b.c IN (1, 3)`) to enable partition pruning while still applying the - * full predicate as a post-scan filter. Note that Spark's own - * [[org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions]] has the same - * limitation for regular partition columns with OR predicates — `(a = 1 OR d = 2)` is - * classified as a data filter, not a partition filter. The difference is that for regular - * columns Spark correctly extracts the pure-partition conjuncts from the top-level AND - * before encountering the OR, whereas for nested columns we may miss some of those. + * Known limitations vs regular partition columns: + * - `(nested_record.level = 'INFO' AND d = 2) OR (nested_record.level = 'ERROR')` is excluded + * entirely (references both partition and data columns). A weaker predicate like + * `nested_record.level IN ('INFO', 'ERROR')` could be extracted but is not implemented. + * Spark has the same OR limitation for regular partition columns. * * @param dataFilters filters to scan for nested partition predicates - * @param partitionColumnNames the set of partition column dot-paths (e.g. Set("nested_record.level")) - * @return only the filters that exclusively reference partition columns + * @param partitionColumnNames partition column dot-paths, e.g. `Set("nested_record.level")` + * @return only the filters whose every column reference is a partition column */ private[hudi] def extractNestedPartitionFilters(dataFilters: Seq[Expression], partitionColumnNames: Set[String]): Seq[Expression] = { @@ -652,26 +571,17 @@ object HoodieFileIndex extends Logging { } /** - * Collects the full dot-paths of outermost [[GetStructField]] chains in an expression tree. - * For `EqualTo(GetStructField(GetStructField(attr("a"), _, "b"), _, "c"), Literal(1))`, - * returns `Seq("a.b.c")` — only the outermost chain, not intermediate segments like "a.b". + * Collects full dot-paths of outermost [[GetStructField]] chains in an expression. + * `EqualTo(a.b.c, 1)` → `Seq("a.b.c")` (not intermediate `"a.b"`). */ private[hudi] def collectOutermostStructFieldPaths(expr: Expression): Seq[String] = { expr match { - case g: GetStructField => - // This is an outermost GetStructField — resolve the full chain and don't recurse - // into children (they are intermediate segments of the same chain). - resolveGetStructFieldPath(g).toSeq - case _ => - // Not a GetStructField — recurse into children to find GetStructField chains. - expr.children.flatMap(collectOutermostStructFieldPaths) + case g: GetStructField => resolveGetStructFieldPath(g).toSeq + case _ => expr.children.flatMap(collectOutermostStructFieldPaths) } } - /** - * Resolves a [[GetStructField]] chain to its full dot-path string. - * E.g. `GetStructField(GetStructField(attr("a"), _, "b"), _, "c")` resolves to `"a.b.c"`. - */ + /** Resolves a [[GetStructField]] chain to its full dot-path: `attr("a").b.c` → `"a.b.c"`. */ private[hudi] def resolveGetStructFieldPath(expr: Expression): Option[String] = expr match { case GetStructField(child: AttributeReference, _, Some(fieldName)) => Some(child.name + "." + fieldName) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 80c78accf5454..fa38d4dc9ea1b 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -258,7 +258,7 @@ class SparkHoodieTableFileIndex(spark: SparkSession, val resolve = spark.sessionState.analyzer.resolver val partitionColumnNames = getPartitionColumns - // Resolve a GetStructField chain to its full dot-path (e.g. "nested_record.level"). + // Resolves GetStructField chain to full dot-path: GetStructField(attr("a"), _, "b") → "a.b" def getFieldPath(expr: Expression): Option[String] = expr match { case a: AttributeReference => Some(a.name) case GetStructField(child, _, Some(fieldName)) => @@ -266,21 +266,18 @@ class SparkHoodieTableFileIndex(spark: SparkSession, case _ => None } - // Returns true if every column reference in the expression resolves to a partition column. - // For flat columns (e.g. "country"), checks AttributeReference.name directly. - // For nested columns (e.g. "nested_record.level"), walks GetStructField chains to build - // the full dot-path and checks whether it matches a partition column name. - // This avoids the overly broad struct-parent prefix matching that would misclassify - // filters on non-partition nested fields (e.g. "nested_record.nested_int") as partition filters. + // True if every column reference in expr resolves to a partition column. + // For nested columns, walks GetStructField chains to match the full dot-path. + // Example: partition = "nested_record.level" + // nested_record.level = 'INFO' → GetStructField path "nested_record.level" → true + // nested_record.nested_int = 10 → GetStructField path "nested_record.nested_int" → false + // IsNotNull(nested_record) → AttributeReference "nested_record" not in partitionColumnNames → false def referencesOnlyPartitionColumns(expr: Expression): Boolean = expr match { case g: GetStructField => getFieldPath(g).exists(path => partitionColumnNames.exists(pc => resolve(path, pc))) - case _: AttributeReference => - // Flat attribute — check if it's a partition column directly - partitionColumnNames.exists(pc => resolve(expr.asInstanceOf[AttributeReference].name, pc)) + case a: AttributeReference => + partitionColumnNames.exists(pc => resolve(a.name, pc)) case _ => - // For compound expressions (And, Or, EqualTo, etc.), all children must reference - // only partition columns. Literals have no children and pass vacuously. expr.children.forall(referencesOnlyPartitionColumns) } @@ -639,33 +636,10 @@ object SparkHoodieTableFileIndex extends SparkAdapterSupport { } /** - * This method unravels [[StructType]] into a [[Map]] of pairs of dot-path notation with corresponding - * [[StructField]] object for every field of the provided [[StructType]], recursively. - * - * For example, following struct - *
-   *   StructType(
-   *     StructField("a",
-   *       StructType(
-   *          StructField("b", StringType),
-   *          StructField("c", IntType)
-   *       )
-   *     )
-   *   )
-   * 
- * - * will be converted into following mapping: - * - *
-   *   "a.b" -> StructField("b", StringType),
-   *   "a.c" -> StructField("c", IntType),
-   * 
- */ - /** - * Builds a map from dot-path field names to [[StructField]]s for all leaf fields in a schema. - * For nested structs, both the key and the [[StructField.name]] use the full dot-path. - * E.g. for schema `StructType(StructField("a", StructType(StructField("b", IntegerType))))`, - * returns `Map("a.b" -> StructField("a.b", IntegerType))`. + * Maps every leaf field in `structType` to its dot-path name. + * Both the key and [[StructField.name]] use the full path. + * E.g. `StructType(StructField("a", StructType(StructField("b", IntegerType))))` + * → `Map("a.b" -> StructField("a.b", IntegerType))`. */ private def generateFieldMap(structType: StructType) : Map[String, StructField] = { def traverse(structField: Either[StructField, StructType]) : Map[String, StructField] = { diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index 7c6bff0be08f6..a1fcd12fc3625 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -44,14 +44,14 @@ import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GreaterThanOrEqual, LessThan, Literal} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GetStructField, GreaterThanOrEqual, LessThan, Literal, Or} import org.apache.spark.sql.execution.datasources.{NoopCache, PartitionDirectory} import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.junit.jupiter.api.{BeforeEach, Test} -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} +import org.junit.jupiter.api.Assertions.{assertEquals, assertThrows, assertTrue} import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.{Arguments, CsvSource, MethodSource, ValueSource} @@ -859,6 +859,113 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } } + // ---- buildNestedPartitionSchema tests ---- + + @Test + def testBuildNestedPartitionSchema_emptySchema(): Unit = { + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(new StructType()) + assertTrue(result.isEmpty) + } + + @Test + def testBuildNestedPartitionSchema_flatColumn(): Unit = { + val flat = StructType(Seq(StructField("country", StringType))) + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + assertEquals(StructType(Seq(StructField("country", StringType, nullable = true))), result) + } + + @Test + def testBuildNestedPartitionSchema_singleNested(): Unit = { + // "nested_record.level" → StructType(nested_record: StructType(level: StringType)) + val flat = StructType(Seq(StructField("nested_record.level", StringType))) + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + assertEquals(1, result.fields.length) + assertEquals("nested_record", result.fields(0).name) + val inner = result.fields(0).dataType.asInstanceOf[StructType] + assertEquals(StructType(Seq(StructField("level", StringType, nullable = true))), inner) + } + + @Test + def testBuildNestedPartitionSchema_twoLevelNesting(): Unit = { + // "a.b.c" → StructType(a: StructType(b: StructType(c: IntegerType))) + val flat = StructType(Seq(StructField("a.b.c", IntegerType))) + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + val a = result.fields(0) + assertEquals("a", a.name) + val b = a.dataType.asInstanceOf[StructType].fields(0) + assertEquals("b", b.name) + val c = b.dataType.asInstanceOf[StructType].fields(0) + assertEquals("c", c.name) + assertEquals(IntegerType, c.dataType) + } + + @Test + def testBuildNestedPartitionSchema_siblingFields(): Unit = { + // "a.b" and "a.c" share parent "a" + val flat = StructType(Seq(StructField("a.b", StringType), StructField("a.c", IntegerType))) + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + assertEquals(1, result.fields.length) + val inner = result.fields(0).dataType.asInstanceOf[StructType] + assertEquals(2, inner.fields.length) + assertEquals("b", inner.fields(0).name) + assertEquals("c", inner.fields(1).name) + } + + @Test + def testBuildNestedPartitionSchema_mixedFlatAndNested(): Unit = { + val flat = StructType(Seq(StructField("country", StringType), StructField("nested_record.level", StringType))) + val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + assertEquals(2, result.fields.length) + assertEquals("country", result.fields(0).name) + assertEquals(StringType, result.fields(0).dataType) + assertEquals("nested_record", result.fields(1).name) + assertTrue(result.fields(1).dataType.isInstanceOf[StructType]) + } + + @Test + def testBuildNestedPartitionSchema_conflictThrows(): Unit = { + // "a" as leaf and "a.b" as nested — conflict + val flat = StructType(Seq(StructField("a", StringType), StructField("a.b", IntegerType))) + assertThrows(classOf[IllegalStateException], () => { + SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) + }) + } + + // ---- extractNestedPartitionFilters tests ---- + + @Test + def testExtractNestedPartitionFilters_correctFilterExtracted(): Unit = { + // nested_record.level = 'INFO' → extracted; int_field = 5 → excluded + val structType = StructType(Seq(StructField("level", StringType))) + val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) + val partFilter = EqualTo(gsf, Literal("INFO")) + val dataFilter = EqualTo(AttributeReference("int_field", IntegerType)(), Literal(5)) + val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(partFilter, dataFilter), Set("nested_record.level")) + assertEquals(1, result.size) + assertEquals(partFilter, result.head) + } + + @Test + def testExtractNestedPartitionFilters_siblingFieldExcluded(): Unit = { + // nested_record.nested_int = 10 should NOT match partition column nested_record.level + val structType = StructType(Seq(StructField("nested_int", IntegerType), StructField("level", StringType))) + val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("nested_int")) + val filter = EqualTo(gsf, Literal(10)) + val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(filter), Set("nested_record.level")) + assertTrue(result.isEmpty) + } + + @Test + def testExtractNestedPartitionFilters_orWithOnlyPartitionColumns(): Unit = { + // nested_record.level = 'INFO' OR nested_record.level = 'ERROR' → extracted + val structType = StructType(Seq(StructField("level", StringType))) + val gsf1 = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) + val gsf2 = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) + val orExpr = Or(EqualTo(gsf1, Literal("INFO")), EqualTo(gsf2, Literal("ERROR"))) + val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(orExpr), Set("nested_record.level")) + assertEquals(1, result.size) + } + } object TestHoodieFileIndex { From 3d862571ca2e1847e946bed38ae8e3462370df6d Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Fri, 10 Apr 2026 21:31:14 -0700 Subject: [PATCH 15/20] fix ci --- .../org/apache/hudi/TestHoodieFileIndex.scala | 24 +++++++++---------- 1 file changed, 12 insertions(+), 12 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index a1fcd12fc3625..d9d4dbc1b691a 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -862,20 +862,20 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS // ---- buildNestedPartitionSchema tests ---- @Test - def testBuildNestedPartitionSchema_emptySchema(): Unit = { + def testBuildNestedPartitionSchemaEmpty(): Unit = { val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(new StructType()) assertTrue(result.isEmpty) } @Test - def testBuildNestedPartitionSchema_flatColumn(): Unit = { + def testBuildNestedPartitionSchemaFlatColumn(): Unit = { val flat = StructType(Seq(StructField("country", StringType))) val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) assertEquals(StructType(Seq(StructField("country", StringType, nullable = true))), result) } @Test - def testBuildNestedPartitionSchema_singleNested(): Unit = { + def testBuildNestedPartitionSchemaSingleNested(): Unit = { // "nested_record.level" → StructType(nested_record: StructType(level: StringType)) val flat = StructType(Seq(StructField("nested_record.level", StringType))) val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) @@ -886,7 +886,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testBuildNestedPartitionSchema_twoLevelNesting(): Unit = { + def testBuildNestedPartitionSchemaTwoLevelNesting(): Unit = { // "a.b.c" → StructType(a: StructType(b: StructType(c: IntegerType))) val flat = StructType(Seq(StructField("a.b.c", IntegerType))) val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) @@ -900,7 +900,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testBuildNestedPartitionSchema_siblingFields(): Unit = { + def testBuildNestedPartitionSchemaSiblingFields(): Unit = { // "a.b" and "a.c" share parent "a" val flat = StructType(Seq(StructField("a.b", StringType), StructField("a.c", IntegerType))) val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) @@ -912,7 +912,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testBuildNestedPartitionSchema_mixedFlatAndNested(): Unit = { + def testBuildNestedPartitionSchemaMixedFlatAndNested(): Unit = { val flat = StructType(Seq(StructField("country", StringType), StructField("nested_record.level", StringType))) val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) assertEquals(2, result.fields.length) @@ -923,18 +923,18 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testBuildNestedPartitionSchema_conflictThrows(): Unit = { + def testBuildNestedPartitionSchemaConflictThrows(): Unit = { // "a" as leaf and "a.b" as nested — conflict val flat = StructType(Seq(StructField("a", StringType), StructField("a.b", IntegerType))) - assertThrows(classOf[IllegalStateException], () => { + assertThrows(classOf[IllegalStateException]) { SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - }) + } } // ---- extractNestedPartitionFilters tests ---- @Test - def testExtractNestedPartitionFilters_correctFilterExtracted(): Unit = { + def testExtractNestedPartitionFiltersCorrectFilterExtracted(): Unit = { // nested_record.level = 'INFO' → extracted; int_field = 5 → excluded val structType = StructType(Seq(StructField("level", StringType))) val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) @@ -946,7 +946,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testExtractNestedPartitionFilters_siblingFieldExcluded(): Unit = { + def testExtractNestedPartitionFiltersSiblingFieldExcluded(): Unit = { // nested_record.nested_int = 10 should NOT match partition column nested_record.level val structType = StructType(Seq(StructField("nested_int", IntegerType), StructField("level", StringType))) val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("nested_int")) @@ -956,7 +956,7 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS } @Test - def testExtractNestedPartitionFilters_orWithOnlyPartitionColumns(): Unit = { + def testExtractNestedPartitionFiltersOrWithOnlyPartitionColumns(): Unit = { // nested_record.level = 'INFO' OR nested_record.level = 'ERROR' → extracted val structType = StructType(Seq(StructField("level", StringType))) val gsf1 = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) From 59d4bab6421194bfb7786f5ba208ac6a2c287c7e Mon Sep 17 00:00:00 2001 From: Lin Liu Date: Sat, 11 Apr 2026 07:58:17 -0700 Subject: [PATCH 16/20] Ci failures --- ...park3HoodiePruneFileSourcePartitions.scala | 2 +- ...estGetPartitionFiltersAndDataFilters.scala | 149 ------------------ ...park4HoodiePruneFileSourcePartitions.scala | 2 +- 3 files changed, 2 insertions(+), 151 deletions(-) delete mode 100644 hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala diff --git a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala index b41b367514b51..3cb9e8250da96 100644 --- a/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark3-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark3HoodiePruneFileSourcePartitions.scala @@ -81,7 +81,7 @@ case class Spark3HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru } -private[analysis] object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { +private object Spark3HoodiePruneFileSourcePartitions extends PredicateHelper { private val exprUtils = sparkAdapter.getCatalystExpressionUtils diff --git a/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala b/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala deleted file mode 100644 index 17bac4cde5287..0000000000000 --- a/hudi-spark-datasource/hudi-spark3-common/src/test/scala/org/apache/spark/sql/hudi/analysis/TestGetPartitionFiltersAndDataFilters.scala +++ /dev/null @@ -1,149 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.spark.sql.hudi.analysis - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ -import org.junit.jupiter.api.Assertions.{assertEquals, assertTrue} -import org.junit.jupiter.api.Test - -/** - * Unit tests for [[Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters]]. - * - * Tests that partition filter classification correctly handles nested partition columns - * using the nested [[StructType]] partition schema from `partitionSchemaForSpark`. - */ -class TestGetPartitionFiltersAndDataFilters { - - private def attr(name: String, dataType: DataType = StringType): AttributeReference = - AttributeReference(name, dataType)() - - // Nested partition schema for partition column "nested_record.level" - // partitionSchemaForSpark produces: StructType(StructField("nested_record", StructType(StructField("level", StringType)))) - private val nestedPartitionSchema = StructType(Seq( - StructField("nested_record", StructType(Seq(StructField("level", StringType)))) - )) - - // Flat partition schema for partition column "country" - private val flatPartitionSchema = StructType(Seq( - StructField("country", StringType) - )) - - @Test - def testFlatPartitionColumnRecognized(): Unit = { - // country = 'US' - val filter = EqualTo(attr("country"), Literal("US")) - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(flatPartitionSchema, Seq(filter)) - assertEquals(1, partFilters.size, "Flat partition filter should be classified as partition filter") - assertTrue(dataFilters.isEmpty, "No data filters expected") - } - - @Test - def testFlatDataColumnExcluded(): Unit = { - // int_field = 5 - val filter = EqualTo(attr("int_field", IntegerType), Literal(5)) - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(flatPartitionSchema, Seq(filter)) - assertTrue(partFilters.isEmpty, "Data column filter should not be a partition filter") - assertEquals(1, dataFilters.size) - } - - @Test - def testNestedPartitionColumnStructRootRecognized(): Unit = { - // For nested partition column "nested_record.level", the nested schema has "nested_record" - // as a top-level field. Spark represents filter nested_record.level = 'INFO' as - // GetStructField(attr("nested_record"), ..., "level") = Literal("INFO"). - // The AttributeReference "nested_record" should be recognized as referencing a partition column. - val nestedRecordType = StructType(Seq(StructField("level", StringType))) - val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("level")) - val filter = EqualTo(gsf, Literal("INFO")) - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) - assertEquals(1, partFilters.size, "Nested partition filter should be classified as partition filter") - } - - @Test - def testNonPartitionNestedFieldExcluded(): Unit = { - // nested_record.nested_int = 10 — same struct root but different field - // In the nested schema, "nested_record" is a top-level name, so the AttributeReference - // "nested_record" still matches. This means the filter IS classified as a partition filter - // by getPartitionFiltersAndDataFilters (which uses AttributeSet-based classification). - // The precise filtering happens downstream in listMatchingPartitionPaths via - // referencesOnlyPartitionColumns, which walks GetStructField chains. - val nestedRecordType = StructType(Seq(StructField("nested_int", IntegerType), StructField("level", StringType))) - val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("nested_int")) - val filter = EqualTo(gsf, Literal(10)) - val (partFilters, _) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) - // Note: getPartitionFiltersAndDataFilters classifies by AttributeReference root ("nested_record"), - // which IS in the nested partition schema. So this is classified as a partition filter here. - // The downstream referencesOnlyPartitionColumns in listMatchingPartitionPaths will correctly - // exclude it by checking the full GetStructField path "nested_record.nested_int". - assertEquals(1, partFilters.size, - "Filter on non-partition nested field is classified as partition filter by AttributeSet matching " + - "(precise exclusion happens downstream in listMatchingPartitionPaths)") - } - - @Test - def testMixedPartitionAndDataColumns(): Unit = { - // Two filters: nested_record.level = 'INFO' and int_field = 5 - val nestedRecordType = StructType(Seq(StructField("level", StringType))) - val gsf = GetStructField(attr("nested_record", nestedRecordType), 0, Some("level")) - val partFilter = EqualTo(gsf, Literal("INFO")) - val dataFilter = EqualTo(attr("int_field", IntegerType), Literal(5)) - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(partFilter, dataFilter)) - assertEquals(1, partFilters.size, "Partition filter should be extracted") - assertEquals(1, dataFilters.size, "Data filter should remain") - } - - @Test - def testIsNotNullOnStructRootClassifiedAsPartition(): Unit = { - // IsNotNull(nested_record) — Spark auto-adds this. - // Since "nested_record" is in the nested partition schema names, it's classified as partition. - val nestedRecordType = StructType(Seq(StructField("level", StringType))) - val filter = IsNotNull(attr("nested_record", nestedRecordType)) - val (partFilters, _) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) - assertEquals(1, partFilters.size, - "IsNotNull on struct root should be classified as partition filter (struct root is in nested schema)") - } - - @Test - def testUnrelatedStructNotRecognized(): Unit = { - // other_struct.field = 'value' — struct root "other_struct" is NOT in partition schema - val otherType = StructType(Seq(StructField("field", StringType))) - val gsf = GetStructField(attr("other_struct", otherType), 0, Some("field")) - val filter = EqualTo(gsf, Literal("value")) - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq(filter)) - assertTrue(partFilters.isEmpty, "Filter on unrelated struct should not be a partition filter") - assertEquals(1, dataFilters.size) - } - - @Test - def testEmptyFilters(): Unit = { - val (partFilters, dataFilters) = - Spark3HoodiePruneFileSourcePartitions.getPartitionFiltersAndDataFilters(nestedPartitionSchema, Seq.empty) - assertTrue(partFilters.isEmpty) - assertTrue(dataFilters.isEmpty) - } -} diff --git a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala index faad50e8706fc..0f6cf87da86f6 100644 --- a/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala +++ b/hudi-spark-datasource/hudi-spark4-common/src/main/scala/org/apache/spark/sql/hudi/analysis/Spark4HoodiePruneFileSourcePartitions.scala @@ -81,7 +81,7 @@ case class Spark4HoodiePruneFileSourcePartitions(spark: SparkSession) extends Ru } -private[analysis] object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { +private object Spark4HoodiePruneFileSourcePartitions extends PredicateHelper { private val exprUtils = sparkAdapter.getCatalystExpressionUtils From 830c281f18fcb400ab4e325f1e87a191e703fe97 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 15 May 2026 13:47:27 -0700 Subject: [PATCH 17/20] Address review comments: replace FQN with imports, parameterize tests - SparkHoodieTableFileIndex: add `DataType` and `LinkedHashMap` imports, drop the inline fully-qualified references in `NestedFieldNode` and `buildNestedPartitionSchema`. - TestCOWDataSource: add imports for `LogicalRelation`, `HadoopFsRelation`, `HoodieFileIndex`, `HoodieBaseRelation`; drop FQN in `runNestedFieldPartitionTest`. - TestHoodieFileIndex: collapse six `testBuildNestedPartitionSchema*` cases into a single `@ParameterizedTest` driven by `buildNestedPartitionSchemaCases`; collapse three `testExtractNestedPartitionFilters*` cases into `extractNestedPartitionFiltersCases`. Keeps the conflict-throws case as a focused `@Test`. --- .../hudi/SparkHoodieTableFileIndex.scala | 12 +- .../org/apache/hudi/TestHoodieFileIndex.scala | 164 ++++++++---------- .../hudi/functional/TestCOWDataSource.scala | 10 +- 3 files changed, 86 insertions(+), 100 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index fa38d4dc9ea1b..69aa6ac1a3e85 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -50,7 +50,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredic import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ByteType, DateType, IntegerType, LongType, ShortType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{ByteType, DataType, DateType, IntegerType, LongType, ShortType, StringType, StructField, StructType} import org.slf4j.LoggerFactory import javax.annotation.concurrent.NotThreadSafe @@ -58,6 +58,8 @@ import javax.annotation.concurrent.NotThreadSafe import java.lang.reflect.{Array => JArray} import java.util.Collections +import scala.collection.mutable.LinkedHashMap + import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.util.{Success, Try} @@ -535,8 +537,8 @@ object SparkHoodieTableFileIndex extends SparkAdapterSupport { private val PUT_LEAF_FILES_METHOD_NAME = "putLeafFiles" private case class NestedFieldNode( - leafType: Option[org.apache.spark.sql.types.DataType], - children: scala.collection.mutable.LinkedHashMap[String, NestedFieldNode] + leafType: Option[DataType], + children: LinkedHashMap[String, NestedFieldNode] ) /** @@ -553,10 +555,10 @@ object SparkHoodieTableFileIndex extends SparkAdapterSupport { if (flatPartitionSchema.isEmpty) { new StructType() } else { - val root = NestedFieldNode(None, scala.collection.mutable.LinkedHashMap.empty) + val root = NestedFieldNode(None, LinkedHashMap.empty) def getOrCreateChild(parent: NestedFieldNode, name: String): NestedFieldNode = { - parent.children.getOrElseUpdate(name, NestedFieldNode(None, scala.collection.mutable.LinkedHashMap.empty)) + parent.children.getOrElseUpdate(name, NestedFieldNode(None, LinkedHashMap.empty)) } flatPartitionSchema.fields.foreach { field => diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala index d9d4dbc1b691a..8d06e257d1787 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieFileIndex.scala @@ -44,7 +44,7 @@ import org.apache.hudi.testutils.HoodieSparkClientTestBase import org.apache.hudi.util.JFunction import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, GetStructField, GreaterThanOrEqual, LessThan, Literal, Or} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, EqualTo, Expression, GetStructField, GreaterThanOrEqual, LessThan, Literal, Or} import org.apache.spark.sql.execution.datasources.{NoopCache, PartitionDirectory} import org.apache.spark.sql.functions.{lit, struct} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension @@ -861,65 +861,10 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS // ---- buildNestedPartitionSchema tests ---- - @Test - def testBuildNestedPartitionSchemaEmpty(): Unit = { - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(new StructType()) - assertTrue(result.isEmpty) - } - - @Test - def testBuildNestedPartitionSchemaFlatColumn(): Unit = { - val flat = StructType(Seq(StructField("country", StringType))) - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - assertEquals(StructType(Seq(StructField("country", StringType, nullable = true))), result) - } - - @Test - def testBuildNestedPartitionSchemaSingleNested(): Unit = { - // "nested_record.level" → StructType(nested_record: StructType(level: StringType)) - val flat = StructType(Seq(StructField("nested_record.level", StringType))) - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - assertEquals(1, result.fields.length) - assertEquals("nested_record", result.fields(0).name) - val inner = result.fields(0).dataType.asInstanceOf[StructType] - assertEquals(StructType(Seq(StructField("level", StringType, nullable = true))), inner) - } - - @Test - def testBuildNestedPartitionSchemaTwoLevelNesting(): Unit = { - // "a.b.c" → StructType(a: StructType(b: StructType(c: IntegerType))) - val flat = StructType(Seq(StructField("a.b.c", IntegerType))) - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - val a = result.fields(0) - assertEquals("a", a.name) - val b = a.dataType.asInstanceOf[StructType].fields(0) - assertEquals("b", b.name) - val c = b.dataType.asInstanceOf[StructType].fields(0) - assertEquals("c", c.name) - assertEquals(IntegerType, c.dataType) - } - - @Test - def testBuildNestedPartitionSchemaSiblingFields(): Unit = { - // "a.b" and "a.c" share parent "a" - val flat = StructType(Seq(StructField("a.b", StringType), StructField("a.c", IntegerType))) - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - assertEquals(1, result.fields.length) - val inner = result.fields(0).dataType.asInstanceOf[StructType] - assertEquals(2, inner.fields.length) - assertEquals("b", inner.fields(0).name) - assertEquals("c", inner.fields(1).name) - } - - @Test - def testBuildNestedPartitionSchemaMixedFlatAndNested(): Unit = { - val flat = StructType(Seq(StructField("country", StringType), StructField("nested_record.level", StringType))) - val result = SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat) - assertEquals(2, result.fields.length) - assertEquals("country", result.fields(0).name) - assertEquals(StringType, result.fields(0).dataType) - assertEquals("nested_record", result.fields(1).name) - assertTrue(result.fields(1).dataType.isInstanceOf[StructType]) + @ParameterizedTest + @MethodSource(Array("buildNestedPartitionSchemaCases")) + def testBuildNestedPartitionSchema(name: String, flat: StructType, expected: StructType): Unit = { + assertEquals(expected, SparkHoodieTableFileIndex.buildNestedPartitionSchema(flat)) } @Test @@ -933,37 +878,13 @@ class TestHoodieFileIndex extends HoodieSparkClientTestBase with ScalaAssertionS // ---- extractNestedPartitionFilters tests ---- - @Test - def testExtractNestedPartitionFiltersCorrectFilterExtracted(): Unit = { - // nested_record.level = 'INFO' → extracted; int_field = 5 → excluded - val structType = StructType(Seq(StructField("level", StringType))) - val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) - val partFilter = EqualTo(gsf, Literal("INFO")) - val dataFilter = EqualTo(AttributeReference("int_field", IntegerType)(), Literal(5)) - val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(partFilter, dataFilter), Set("nested_record.level")) - assertEquals(1, result.size) - assertEquals(partFilter, result.head) - } - - @Test - def testExtractNestedPartitionFiltersSiblingFieldExcluded(): Unit = { - // nested_record.nested_int = 10 should NOT match partition column nested_record.level - val structType = StructType(Seq(StructField("nested_int", IntegerType), StructField("level", StringType))) - val gsf = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("nested_int")) - val filter = EqualTo(gsf, Literal(10)) - val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(filter), Set("nested_record.level")) - assertTrue(result.isEmpty) - } - - @Test - def testExtractNestedPartitionFiltersOrWithOnlyPartitionColumns(): Unit = { - // nested_record.level = 'INFO' OR nested_record.level = 'ERROR' → extracted - val structType = StructType(Seq(StructField("level", StringType))) - val gsf1 = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) - val gsf2 = GetStructField(AttributeReference("nested_record", structType)(), 0, Some("level")) - val orExpr = Or(EqualTo(gsf1, Literal("INFO")), EqualTo(gsf2, Literal("ERROR"))) - val result = HoodieFileIndex.extractNestedPartitionFilters(Seq(orExpr), Set("nested_record.level")) - assertEquals(1, result.size) + @ParameterizedTest + @MethodSource(Array("extractNestedPartitionFiltersCases")) + def testExtractNestedPartitionFilters(name: String, + filters: Seq[Expression], + partitionColumns: Set[String], + expected: Seq[Expression]): Unit = { + assertEquals(expected, HoodieFileIndex.extractNestedPartitionFilters(filters, partitionColumns)) } } @@ -978,4 +899,65 @@ object TestHoodieFileIndex { Arguments.arguments("org.apache.hudi.keygen.TimestampBasedKeyGenerator") ) } + + def buildNestedPartitionSchemaCases(): java.util.stream.Stream[Arguments] = { + val nested = StructType(Seq( + StructField("nested_record", StructType(Seq(StructField("level", StringType, nullable = true))), nullable = true))) + val twoLevel = StructType(Seq( + StructField("a", StructType(Seq( + StructField("b", StructType(Seq(StructField("c", IntegerType, nullable = true))), nullable = true))), nullable = true))) + val siblings = StructType(Seq( + StructField("a", StructType(Seq( + StructField("b", StringType, nullable = true), + StructField("c", IntegerType, nullable = true))), nullable = true))) + val mixed = StructType(Seq( + StructField("country", StringType, nullable = true), + StructField("nested_record", StructType(Seq(StructField("level", StringType, nullable = true))), nullable = true))) + java.util.stream.Stream.of( + Arguments.of("empty", + new StructType(), + new StructType()), + Arguments.of("flat", + StructType(Seq(StructField("country", StringType))), + StructType(Seq(StructField("country", StringType, nullable = true)))), + Arguments.of("singleNested", + StructType(Seq(StructField("nested_record.level", StringType))), + nested), + Arguments.of("twoLevelNesting", + StructType(Seq(StructField("a.b.c", IntegerType))), + twoLevel), + Arguments.of("siblingFields", + StructType(Seq(StructField("a.b", StringType), StructField("a.c", IntegerType))), + siblings), + Arguments.of("mixedFlatAndNested", + StructType(Seq(StructField("country", StringType), StructField("nested_record.level", StringType))), + mixed) + ) + } + + def extractNestedPartitionFiltersCases(): java.util.stream.Stream[Arguments] = { + val levelStruct = StructType(Seq(StructField("level", StringType))) + val multiFieldStruct = StructType(Seq( + StructField("nested_int", IntegerType), StructField("level", StringType))) + + val partFilter = EqualTo( + GetStructField(AttributeReference("nested_record", levelStruct)(), 0, Some("level")), + Literal("INFO")) + val dataFilter = EqualTo(AttributeReference("int_field", IntegerType)(), Literal(5)) + val siblingFilter = EqualTo( + GetStructField(AttributeReference("nested_record", multiFieldStruct)(), 0, Some("nested_int")), + Literal(10)) + val orFilter = Or( + EqualTo(GetStructField(AttributeReference("nested_record", levelStruct)(), 0, Some("level")), Literal("INFO")), + EqualTo(GetStructField(AttributeReference("nested_record", levelStruct)(), 0, Some("level")), Literal("ERROR"))) + + java.util.stream.Stream.of( + Arguments.of("partitionFilterExtractedDataFilterDropped", + Seq(partFilter, dataFilter), Set("nested_record.level"), Seq(partFilter)), + Arguments.of("siblingFieldExcluded", + Seq(siblingFilter), Set("nested_record.level"), Seq.empty[Expression]), + Arguments.of("orWithOnlyPartitionColumnsExtracted", + Seq(orFilter), Set("nested_record.level"), Seq(orFilter)) + ) + } } diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 84470ef2c730b..413f273ceebe7 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -46,9 +46,11 @@ import org.apache.hudi.storage.{HoodieStorage, StoragePath, StoragePathFilter} import org.apache.hudi.table.HoodieSparkTable import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieSparkClientTestBase} import org.apache.hudi.util.JFunction +import org.apache.hudi.{HoodieBaseRelation, HoodieFileIndex} import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.{DataFrame, DataFrameWriter, Dataset, Encoders, Row, SaveMode, SparkSession, SparkSessionExtensions} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions.{col, concat, lit, udf, when} import org.apache.spark.sql.hudi.HoodieSparkSessionExtension import org.apache.spark.sql.types.{ArrayType, DataTypes, DateType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} @@ -2740,13 +2742,13 @@ object TestCOWDataSource { // VERIFICATION 1: Check partition schema contains the nested field val snapshotRelation = snapshotDF.queryExecution.optimizedPlan.collectFirst { - case lr: org.apache.spark.sql.execution.datasources.LogicalRelation => lr + case lr: LogicalRelation => lr } assertTrue(snapshotRelation.isDefined, s"LogicalRelation should exist for $tableType") val fileIndex = snapshotRelation.get.relation match { - case fsRelation: org.apache.spark.sql.execution.datasources.HadoopFsRelation => - fsRelation.location.asInstanceOf[org.apache.hudi.HoodieFileIndex] - case baseRelation: org.apache.hudi.HoodieBaseRelation => + case fsRelation: HadoopFsRelation => + fsRelation.location.asInstanceOf[HoodieFileIndex] + case baseRelation: HoodieBaseRelation => baseRelation.fileIndex case _ => null } From 3f38ecc6d897109a47d00b35e3dade3fbc0c3430 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 15 May 2026 13:57:00 -0700 Subject: [PATCH 18/20] Fix imports --- .../scala/org/apache/hudi/SparkHoodieTableFileIndex.scala | 7 +++---- .../org/apache/hudi/functional/TestCOWDataSource.scala | 3 +-- 2 files changed, 4 insertions(+), 6 deletions(-) diff --git a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala index 69aa6ac1a3e85..b80eb204823af 100644 --- a/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala +++ b/hudi-spark-datasource/hudi-spark-common/src/main/scala/org/apache/hudi/SparkHoodieTableFileIndex.scala @@ -36,7 +36,7 @@ import org.apache.hudi.hadoop.HoodieLatestBaseFilesPathFilter import org.apache.hudi.hadoop.fs.HadoopFSUtils import org.apache.hudi.internal.schema.Types.RecordType import org.apache.hudi.internal.schema.utils.Conversions -import org.apache.hudi.keygen.{CustomAvroKeyGenerator, CustomKeyGenerator, StringPartitionPathFormatter, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} +import org.apache.hudi.keygen.{StringPartitionPathFormatter, TimestampBasedAvroKeyGenerator, TimestampBasedKeyGenerator} import org.apache.hudi.metadata.{CatalogBackedTableMetadata, HoodieTableMetadata} import org.apache.hudi.storage.{StoragePath, StoragePathInfo} import org.apache.hudi.util.JFunction @@ -46,7 +46,7 @@ import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, InterpretedPredicate, Literal} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, BasePredicate, BoundReference, EmptyRow, EqualTo, Expression, GetStructField, Literal} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.datasources.{FileStatusCache, NoopCache} import org.apache.spark.sql.internal.SQLConf @@ -58,9 +58,8 @@ import javax.annotation.concurrent.NotThreadSafe import java.lang.reflect.{Array => JArray} import java.util.Collections -import scala.collection.mutable.LinkedHashMap - import scala.collection.JavaConverters._ +import scala.collection.mutable.LinkedHashMap import scala.language.implicitConversions import scala.util.{Success, Try} diff --git a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala index 413f273ceebe7..106c1f09a7ae2 100644 --- a/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala +++ b/hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala @@ -17,7 +17,7 @@ package org.apache.hudi.functional -import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSchemaConversionUtils, HoodieSparkUtils, QuickstartUtils, ScalaAssertionSupport} +import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieBaseRelation, HoodieDataSourceHelpers, HoodieFileIndex, HoodieSchemaConversionUtils, HoodieSparkUtils, QuickstartUtils, ScalaAssertionSupport} import org.apache.hudi.DataSourceWriteOptions.{INLINE_CLUSTERING_ENABLE, KEYGENERATOR_CLASS_NAME} import org.apache.hudi.HoodieConversionUtils.toJavaOption import org.apache.hudi.QuickstartUtils.{convertToStringList, getQuickstartWriteConfigs} @@ -46,7 +46,6 @@ import org.apache.hudi.storage.{HoodieStorage, StoragePath, StoragePathFilter} import org.apache.hudi.table.HoodieSparkTable import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieSparkClientTestBase} import org.apache.hudi.util.JFunction -import org.apache.hudi.{HoodieBaseRelation, HoodieFileIndex} import org.apache.hadoop.fs.FileSystem import org.apache.spark.sql.{DataFrame, DataFrameWriter, Dataset, Encoders, Row, SaveMode, SparkSession, SparkSessionExtensions} From 525f93249b7d1ee56e9cfb1b67be3fc5f97c3a23 Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 15 May 2026 14:11:36 -0700 Subject: [PATCH 19/20] Fix build --- .../hudi/utilities/HiveIncrementalPuller.java | 19 +++++++++++-------- 1 file changed, 11 insertions(+), 8 deletions(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java index fede1b8fba030..37009c04e651f 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java @@ -20,9 +20,9 @@ import org.apache.hudi.common.table.HoodieTableMetaClient; import org.apache.hudi.common.table.timeline.HoodieInstant; -import org.apache.hudi.common.util.FileIOUtils; import org.apache.hudi.common.util.Option; import org.apache.hudi.exception.HoodieException; +import org.apache.hudi.hadoop.fs.HadoopFSUtils; import org.apache.hudi.utilities.exception.HoodieIncrementalPullException; import org.apache.hudi.utilities.exception.HoodieIncrementalPullSQLException; @@ -50,6 +50,8 @@ import java.util.Scanner; import java.util.stream.Collectors; +import static org.apache.hudi.io.util.FileIOUtils.readAsUTFString; + /** * Utility to pull data after a given commit, based on the supplied HiveQL and save the delta as another hive temporary * table. This temporary table can be further read using {@link org.apache.hudi.utilities.sources.HiveIncrPullSource} and the changes can @@ -115,7 +117,7 @@ public HiveIncrementalPuller(Config config) throws IOException { this.config = config; validateConfig(config); String templateContent = - FileIOUtils.readAsUTFString(this.getClass().getResourceAsStream("/IncrementalPull.sqltemplate")); + readAsUTFString(this.getClass().getResourceAsStream("/IncrementalPull.sqltemplate")); incrementalPullSQLTemplate = new ST(templateContent); } @@ -303,7 +305,7 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO Option lastCommit = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant(); if (lastCommit.isPresent()) { - return lastCommit.get().getTimestamp(); + return lastCommit.get().requestedTime(); } return "0"; } @@ -331,14 +333,15 @@ private boolean ensureTempPathExists(FileSystem fs, String lastCommitTime) throw } private String getLastCommitTimePulled(FileSystem fs, String sourceTableLocation) { - HoodieTableMetaClient metadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(sourceTableLocation).build(); + HoodieTableMetaClient metadata = HoodieTableMetaClient.builder() + .setConf(HadoopFSUtils.getStorageConfWithCopy(fs.getConf())) + .setBasePath(sourceTableLocation).build(); List commitsToSync = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants() - .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstantsAsStream().map(HoodieInstant::getTimestamp) + .findInstantsAfter(config.fromCommitTime, config.maxCommits).getInstantsAsStream().map(HoodieInstant::requestedTime) .collect(Collectors.toList()); if (commitsToSync.isEmpty()) { - LOG.info("Nothing to sync. All commits in {} are {} and from commit time is {}", config.sourceTable, - metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().getInstants(), - config.fromCommitTime); + LOG.info("Nothing to sync. All commits in {} are {} and from commit time is {}", config.sourceTable, metadata.getActiveTimeline().getCommitsTimeline() + .filterCompletedInstants().getInstants(), config.fromCommitTime); return null; } LOG.info("Syncing commits {}", commitsToSync); From 4bc966c2fc48679ffd829f8153a3645765d956ed Mon Sep 17 00:00:00 2001 From: Y Ethan Guo Date: Fri, 15 May 2026 14:15:58 -0700 Subject: [PATCH 20/20] Fix one more build issue --- .../java/org/apache/hudi/utilities/HiveIncrementalPuller.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java index 37009c04e651f..2510edce72a8c 100644 --- a/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java +++ b/hudi-utilities/src/main/java/org/apache/hudi/utilities/HiveIncrementalPuller.java @@ -300,7 +300,8 @@ private String scanForCommitTime(FileSystem fs, String targetDataPath) throws IO if (!fs.exists(new Path(targetDataPath)) || !fs.exists(new Path(targetDataPath + "/.hoodie"))) { return "0"; } - HoodieTableMetaClient metadata = HoodieTableMetaClient.builder().setConf(fs.getConf()).setBasePath(targetDataPath).build(); + HoodieTableMetaClient metadata = HoodieTableMetaClient.builder() + .setConf(HadoopFSUtils.getStorageConfWithCopy(fs.getConf())).setBasePath(targetDataPath).build(); Option lastCommit = metadata.getActiveTimeline().getCommitsTimeline().filterCompletedInstants().lastInstant();