1
0

[HUDI-3935] Adding config to fallback to enabled Partition Values extraction from Partition path (#5377)

This commit is contained in:
Alexey Kudinkin
2022-04-21 01:36:19 -07:00
committed by GitHub
parent a9506aa545
commit 4b296f79cc
12 changed files with 133 additions and 61 deletions

View File

@@ -749,7 +749,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testCopyOnWriteWithDropPartitionColumns(enableDropPartitionColumns: Boolean) {
def testCopyOnWriteWithDroppedPartitionColumns(enableDropPartitionColumns: Boolean) {
val records1 = recordsToStrings(dataGen.generateInsertsContainsAllPartitions("000", 100)).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
@@ -900,7 +900,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
@ParameterizedTest
@ValueSource(booleans = Array(true, false))
def testHoodieBaseFileOnlyViewRelation(useGlobbing: Boolean): Unit = {
def testPartitionColumnsProperHandling(useGlobbing: Boolean): Unit = {
val _spark = spark
import _spark.implicits._
@@ -935,18 +935,41 @@ class TestCOWDataSource extends HoodieClientTestBase {
basePath
}
val res = spark.read.format("hudi").load(path)
// Case #1: Partition columns are read from the data file
val firstDF = spark.read.format("hudi").load(path)
assert(res.count() == 2)
assert(firstDF.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.toSeq,
Seq("2018-09-23", "2018-09-24")
Seq("2018-09-23", "2018-09-24"),
firstDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq
)
assertEquals(
res.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq,
Seq("2018/09/23", "2018/09/24")
Seq("2018/09/23", "2018/09/24"),
firstDF.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq
)
// Case #2: Partition columns are extracted from the partition path
//
// NOTE: This case is only relevant when globbing is NOT used, since when globbing is used Spark
// won't be able to infer partitioning properly
if (!useGlobbing) {
val secondDF = spark.read.format("hudi")
.option(DataSourceReadOptions.EXTRACT_PARTITION_VALUES_FROM_PARTITION_PATH.key, "true")
.load(path)
assert(secondDF.count() == 2)
// data_date is the partition field. Persist to the parquet file using the origin values, and read it.
assertEquals(
Seq("2018/09/23", "2018/09/24"),
secondDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq
)
assertEquals(
Seq("2018/09/23", "2018/09/24"),
secondDF.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq
)
}
}
}