[HUDI-3469] Refactor HoodieTestDataGenerator to provide for reproducible Builds (#4866)
This commit is contained in:
@@ -30,10 +30,8 @@ import org.apache.hudi.keygen.constant.KeyGeneratorOptions.Config
|
||||
import org.apache.hudi.keygen.{ComplexKeyGenerator, TimestampBasedKeyGenerator}
|
||||
import org.apache.hudi.testutils.SparkClientFunctionalTestHarness
|
||||
import org.apache.hudi.{DataSourceReadOptions, DataSourceWriteOptions, HoodieDataSourceHelpers}
|
||||
|
||||
import org.apache.spark.sql._
|
||||
import org.apache.spark.sql.functions.{col, lit}
|
||||
|
||||
import org.junit.jupiter.api.Assertions.{assertEquals, assertFalse, assertTrue}
|
||||
import org.junit.jupiter.api.Tag
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
@@ -60,9 +58,14 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
val updatedVerificationVal: String = "driver_update"
|
||||
|
||||
@ParameterizedTest
|
||||
@CsvSource(Array("true,org.apache.hudi.keygen.SimpleKeyGenerator", "true,org.apache.hudi.keygen.ComplexKeyGenerator",
|
||||
"true,org.apache.hudi.keygen.TimestampBasedKeyGenerator", "false,org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
"false,org.apache.hudi.keygen.ComplexKeyGenerator", "false,org.apache.hudi.keygen.TimestampBasedKeyGenerator"))
|
||||
@CsvSource(Array(
|
||||
"true,org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
"true,org.apache.hudi.keygen.ComplexKeyGenerator",
|
||||
"true,org.apache.hudi.keygen.TimestampBasedKeyGenerator",
|
||||
"false,org.apache.hudi.keygen.SimpleKeyGenerator",
|
||||
"false,org.apache.hudi.keygen.ComplexKeyGenerator",
|
||||
"false,org.apache.hudi.keygen.TimestampBasedKeyGenerator"
|
||||
))
|
||||
def testCopyOnWriteStorage(isMetadataEnabled: Boolean, keyGenClass: String): Unit = {
|
||||
commonOpts += DataSourceWriteOptions.KEYGENERATOR_CLASS_NAME.key() -> keyGenClass
|
||||
if (classOf[ComplexKeyGenerator].getName.equals(keyGenClass)) {
|
||||
@@ -74,7 +77,7 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
commonOpts += Config.TIMESTAMP_TYPE_FIELD_PROP -> "EPOCHMILLISECONDS"
|
||||
commonOpts += Config.TIMESTAMP_OUTPUT_DATE_FORMAT_PROP -> "yyyyMMdd"
|
||||
}
|
||||
val dataGen = new HoodieTestDataGenerator()
|
||||
val dataGen = new HoodieTestDataGenerator(0xDEED)
|
||||
val fs = FSUtils.getFs(basePath, spark.sparkContext.hadoopConfiguration)
|
||||
// Insert Operation
|
||||
val records0 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
|
||||
@@ -101,9 +104,13 @@ class TestCOWDataSourceStorage extends SparkClientFunctionalTestHarness {
|
||||
var updateDf: DataFrame = null
|
||||
if (classOf[TimestampBasedKeyGenerator].getName.equals(keyGenClass)) {
|
||||
// update current_ts to be same as original record so that partition path does not change with timestamp based key gen
|
||||
val originalRow = inputDF1.filter(col("_row_key") === verificationRowKey).collectAsList().get(0)
|
||||
updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
.withColumn("current_ts", lit(originalRow.getAs("current_ts")))
|
||||
val originalRow = snapshotDF1.filter(col("_row_key") === verificationRowKey).collectAsList().get(0)
|
||||
updateDf = inputDF1.filter(col("_row_key") === verificationRowKey)
|
||||
.withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
.withColumn("current_ts", lit(originalRow.getAs[Long]("current_ts")))
|
||||
.limit(1)
|
||||
val updatedRow = updateDf.collectAsList().get(0)
|
||||
assertEquals(originalRow.getAs[Long]("current_ts"), updatedRow.getAs[Long]("current_ts"));
|
||||
} else {
|
||||
updateDf = snapshotDF1.filter(col("_row_key") === verificationRowKey).withColumn(verificationCol, lit(updatedVerificationVal))
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user