[HUDI-4161] Make sure partition values are taken from partition path (#5699)
This commit is contained in:
@@ -144,7 +144,7 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
def testPrunePartitionForTimestampBasedKeyGenerator(): Unit = {
|
||||
val options = commonOpts ++ Map(
|
||||
"hoodie.compact.inline" -> "false",
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL,
|
||||
DataSourceWriteOptions.TABLE_TYPE.key -> DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL,
|
||||
DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key -> "org.apache.hudi.keygen.TimestampBasedKeyGenerator",
|
||||
Config.TIMESTAMP_TYPE_FIELD_PROP -> "DATE_STRING",
|
||||
Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyy/MM/dd",
|
||||
@@ -176,8 +176,11 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
|
||||
// snapshot query
|
||||
val snapshotQueryRes = spark.read.format("hudi").load(basePath)
|
||||
assertEquals(snapshotQueryRes.where("partition = '2022-01-01'").count, 20)
|
||||
assertEquals(snapshotQueryRes.where("partition = '2022-01-02'").count, 30)
|
||||
// TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10
|
||||
//assertEquals(snapshotQueryRes.where("partition = '2022-01-01'").count, 20)
|
||||
//assertEquals(snapshotQueryRes.where("partition = '2022-01-02'").count, 30)
|
||||
assertEquals(snapshotQueryRes.where("partition = '2022/01/01'").count, 20)
|
||||
assertEquals(snapshotQueryRes.where("partition = '2022/01/02'").count, 30)
|
||||
|
||||
// incremental query
|
||||
val incrementalQueryRes = spark.read.format("hudi")
|
||||
@@ -961,10 +964,14 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
assert(firstDF.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"),
|
||||
firstDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq
|
||||
)
|
||||
// TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10
|
||||
val expectedValues = if (useGlobbing) {
|
||||
Seq("2018-09-23", "2018-09-24")
|
||||
} else {
|
||||
Seq("2018/09/23", "2018/09/24")
|
||||
}
|
||||
|
||||
assertEquals(expectedValues, firstDF.select("data_date").map(_.get(0).toString).collect().sorted.toSeq)
|
||||
assertEquals(
|
||||
Seq("2018/09/23", "2018/09/24"),
|
||||
firstDF.select("_hoodie_partition_path").map(_.get(0).toString).collect().sorted.toSeq
|
||||
|
||||
@@ -17,11 +17,10 @@
|
||||
|
||||
package org.apache.hudi.functional
|
||||
|
||||
import org.apache.avro.generic.GenericRecord
|
||||
import org.apache.hadoop.fs.Path
|
||||
import org.apache.hudi.DataSourceWriteOptions._
|
||||
import org.apache.hudi.common.config.HoodieMetadataConfig
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieRecord, HoodieRecordPayload, HoodieTableType}
|
||||
import org.apache.hudi.common.model.{DefaultHoodieRecordPayload, HoodieTableType}
|
||||
import org.apache.hudi.common.table.HoodieTableMetaClient
|
||||
import org.apache.hudi.common.testutils.HoodieTestDataGenerator
|
||||
import org.apache.hudi.common.testutils.RawTripTestPayload.recordsToStrings
|
||||
@@ -30,9 +29,8 @@ import org.apache.hudi.index.HoodieIndex.IndexType
|
||||
import org.apache.hudi.keygen.NonpartitionedKeyGenerator
|
||||
import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
|
||||
import org.apache.hudi.testutils.{DataSourceTestUtils, HoodieClientTestBase}
|
||||
import org.apache.hudi.{AvroConversionUtils, DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, HoodieSparkUtils, SparkDatasetMixin}
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers, SparkDatasetMixin}
|
||||
import org.apache.log4j.LogManager
|
||||
import org.apache.spark.rdd.RDD
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions._
|
||||
import org.apache.spark.sql.types.BooleanType
|
||||
@@ -41,7 +39,6 @@ import org.junit.jupiter.api.{AfterEach, BeforeEach, Test}
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.CsvSource
|
||||
|
||||
import java.util
|
||||
import scala.collection.JavaConversions._
|
||||
import scala.collection.JavaConverters._
|
||||
|
||||
@@ -864,8 +861,11 @@ class TestMORDataSource extends HoodieClientTestBase with SparkDatasetMixin {
|
||||
val readOptimizedQueryRes = spark.read.format("hudi")
|
||||
.option(DataSourceReadOptions.QUERY_TYPE.key, DataSourceReadOptions.QUERY_TYPE_READ_OPTIMIZED_OPT_VAL)
|
||||
.load(basePath)
|
||||
assertEquals(readOptimizedQueryRes.where("partition = '2022-01-01'").count, 50)
|
||||
assertEquals(readOptimizedQueryRes.where("partition = '2022-01-02'").count, 60)
|
||||
// TODO(HUDI-3204) we have to revert this to pre-existing behavior from 0.10
|
||||
//assertEquals(readOptimizedQueryRes.where("partition = '2022-01-01'").count, 50)
|
||||
//assertEquals(readOptimizedQueryRes.where("partition = '2022-01-02'").count, 60)
|
||||
assertEquals(readOptimizedQueryRes.where("partition = '2022/01/01'").count, 50)
|
||||
assertEquals(readOptimizedQueryRes.where("partition = '2022/01/02'").count, 60)
|
||||
|
||||
// incremental query
|
||||
val incrementalQueryRes = spark.read.format("hudi")
|
||||
|
||||
Reference in New Issue
Block a user