1
0

[HUDI-3469] Refactor HoodieTestDataGenerator to provide for reproducible Builds (#4866)

This commit is contained in:
Alexey Kudinkin
2022-03-01 22:15:26 -08:00
committed by GitHub
parent 3b2da9f138
commit 85f47b53df
13 changed files with 159 additions and 115 deletions

View File

@@ -257,7 +257,8 @@ object HoodieSparkSqlWriter {
DataSourceWriteOptions.KEYGENERATOR_CONSISTENT_LOGICAL_TIMESTAMP_ENABLED.defaultValue()).toBoolean)
.asInstanceOf[Comparable[_]]
DataSourceUtils.createHoodieRecord(processedRecord,
orderingVal, keyGenerator.getKey(gr),
orderingVal,
keyGenerator.getKey(gr),
hoodieConfig.getString(PAYLOAD_CLASS_NAME))
} else {
DataSourceUtils.createHoodieRecord(processedRecord, keyGenerator.getKey(gr), hoodieConfig.getString(PAYLOAD_CLASS_NAME))

View File

@@ -104,7 +104,6 @@ import java.util.stream.StreamSupport;
import static java.util.stream.Collectors.mapping;
import static java.util.stream.Collectors.toList;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord;
import static org.apache.spark.sql.functions.callUDF;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -564,8 +563,7 @@ public class TestBootstrap extends HoodieClientTestBase {
final List<String> records = new ArrayList<>();
IntStream.range(from, to).forEach(i -> {
String id = "" + i;
records.add(generateGenericRecord("trip_" + id, Long.toString(timestamp), "rider_" + id, "driver_" + id,
timestamp, false, false).toString());
records.add(new HoodieTestDataGenerator().generateGenericRecord("trip_" + id, Long.toString(timestamp), "rider_" + id, "driver_" + id, timestamp, false, false).toString());
});
if (isPartitioned) {
sqlContext.udf().register("partgen",

View File

@@ -98,7 +98,6 @@ import java.util.stream.StreamSupport;
import static java.util.stream.Collectors.mapping;
import static java.util.stream.Collectors.toList;
import static org.apache.hudi.common.testutils.HoodieTestDataGenerator.generateGenericRecord;
import static org.apache.spark.sql.functions.callUDF;
import static org.junit.jupiter.api.Assertions.assertEquals;
import static org.junit.jupiter.api.Assertions.assertFalse;
@@ -476,8 +475,7 @@ public class TestOrcBootstrap extends HoodieClientTestBase {
final List<String> records = new ArrayList<>();
IntStream.range(from, to).forEach(i -> {
String id = "" + i;
records.add(generateGenericRecord("trip_" + id, Long.toString(timestamp), "rider_" + id, "driver_" + id,
timestamp, false, false).toString());
records.add(new HoodieTestDataGenerator().generateGenericRecord("trip_" + id, Long.toString(timestamp), "rider_" + id, "driver_" + id, timestamp, false, false).toString());
});
if (isPartitioned) {
sqlContext.udf().register("partgen",

View File

@@ -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))
}