[HUDI-3204] Fixing partition-values being derived from partition-path instead of source columns (#5364)
- Scaffolded `Spark24HoodieParquetFileFormat` extending `ParquetFileFormat` and overriding the behavior of adding partition columns to every row - Amended `SparkAdapter`s `createHoodieParquetFileFormat` API to be able to configure whether to append partition values or not - Fallback to append partition values in cases when the source columns are not persisted in data-file - Fixing HoodieBaseRelation incorrectly handling mandatory columns
This commit is contained in:
@@ -747,7 +747,8 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assertEquals(resultSchema, schema1)
|
||||
}
|
||||
|
||||
@ParameterizedTest @ValueSource(booleans = Array(true, false))
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testCopyOnWriteWithDropPartitionColumns(enableDropPartitionColumns: Boolean) {
|
||||
val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 100)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
@@ -897,9 +898,9 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
readResult.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","))
|
||||
}
|
||||
|
||||
@Disabled("HUDI-3204")
|
||||
@Test
|
||||
def testHoodieBaseFileOnlyViewRelation(): Unit = {
|
||||
@ParameterizedTest
|
||||
@ValueSource(booleans = Array(true, false))
|
||||
def testHoodieBaseFileOnlyViewRelation(useGlobbing: Boolean): Unit = {
|
||||
val _spark = spark
|
||||
import _spark.implicits._
|
||||
|
||||
@@ -925,18 +926,27 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
.mode(org.apache.spark.sql.SaveMode.Append)
|
||||
.save(basePath)
|
||||
|
||||
val res = spark.read.format("hudi").load(basePath)
|
||||
// NOTE: We're testing here that both paths are appropriately handling
|
||||
// partition values, regardless of whether we're reading the table
|
||||
// t/h a globbed path or not
|
||||
val path = if (useGlobbing) {
|
||||
s"$basePath/*/*/*/*"
|
||||
} else {
|
||||
basePath
|
||||
}
|
||||
|
||||
val res = spark.read.format("hudi").load(path)
|
||||
|
||||
assert(res.count() == 2)
|
||||
|
||||
// data_date is the partition field. Persist to the parquet file using the origin values, and read it.
|
||||
assertEquals(
|
||||
res.select("data_date").map(_.get(0).toString).collect().sorted,
|
||||
Array("2018-09-23", "2018-09-24")
|
||||
res.select("data_date").map(_.get(0).toString).collect().sorted.toSeq,
|
||||
Seq("2018-09-23", "2018-09-24")
|
||||
)
|
||||
assertEquals(
|
||||
res.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted,
|
||||
Array("2018/09/23", "2018/09/24")
|
||||
res.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq,
|
||||
Seq("2018/09/23", "2018/09/24")
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -57,7 +57,6 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
val verificationCol: String = "driver"
|
||||
val updatedVerificationVal: String = "driver_update"
|
||||
|
||||
@Disabled("HUDI-3896")
|
||||
@ParameterizedTest
|
||||
@CsvSource(Array(
|
||||
"true,org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
|
||||
Reference in New Issue
Block a user