Skip to content

Commit f931c12

Browse files
[MINOR] Reduce UT spark-datasource test times (apache#10547)
* [MINOR] Reduce UT spark-datasource test times * Reverting the parallelism change
1 parent 2042c15 commit f931c12

File tree

4 files changed

+75
-114
lines changed

4 files changed

+75
-114
lines changed

hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/TestHoodieSparkSqlWriter.scala

+24-27
Original file line numberDiff line numberDiff line change
@@ -702,15 +702,11 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
702702
*/
703703
@ParameterizedTest
704704
@CsvSource(value = Array(
705-
"COPY_ON_WRITE,true",
706-
"COPY_ON_WRITE,false",
707-
"MERGE_ON_READ,true",
708-
"MERGE_ON_READ,false"
705+
"COPY_ON_WRITE",
706+
"MERGE_ON_READ"
709707
))
710-
def testSchemaEvolutionForTableType(tableType: String, allowColumnDrop: Boolean): Unit = {
711-
val opts = getCommonParams(tempPath, hoodieFooTableName, tableType) ++ Map(
712-
HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key -> allowColumnDrop.toString
713-
)
708+
def testSchemaEvolutionForTableType(tableType: String): Unit = {
709+
val opts = getCommonParams(tempPath, hoodieFooTableName, tableType)
714710

715711
// Create new table
716712
// NOTE: We disable Schema Reconciliation by default (such that Writer's
@@ -801,28 +797,30 @@ def testBulkInsertForDropPartitionColumn(): Unit = {
801797

802798
val df5 = spark.createDataFrame(sc.parallelize(recordsSeq), structType)
803799

804-
if (allowColumnDrop) {
805-
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, noReconciliationOpts, df5)
806-
807-
val snapshotDF5 = spark.read.format("org.apache.hudi")
808-
.load(tempBasePath + "/*/*/*/*")
809-
810-
assertEquals(35, snapshotDF5.count())
800+
// assert error is thrown when dropping is not allowed
801+
val disallowOpts = noReconciliationOpts ++ Map(
802+
HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key -> false.toString
803+
)
804+
assertThrows[SchemaCompatibilityException] {
805+
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, disallowOpts, df5)
806+
}
811807

812-
assertEquals(df5.intersect(dropMetaFields(snapshotDF5)).except(df5).count, 0)
808+
// passes when allowed.
809+
val allowOpts = noReconciliationOpts ++ Map(
810+
HoodieWriteConfig.SCHEMA_ALLOW_AUTO_EVOLUTION_COLUMN_DROP.key -> true.toString
811+
)
812+
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, allowOpts, df5)
813813

814-
val fifthBatchActualSchema = fetchActualSchema()
815-
val fifthBatchExpectedSchema = {
816-
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(hoodieFooTableName)
817-
AvroConversionUtils.convertStructTypeToAvroSchema(df5.schema, structName, nameSpace)
818-
}
814+
val snapshotDF5 = spark.read.format("org.apache.hudi").load(tempBasePath + "/*/*/*/*")
815+
assertEquals(35, snapshotDF5.count())
816+
assertEquals(df5.intersect(dropMetaFields(snapshotDF5)).except(df5).count, 0)
819817

820-
assertEquals(fifthBatchExpectedSchema, fifthBatchActualSchema)
821-
} else {
822-
assertThrows[SchemaCompatibilityException] {
823-
HoodieSparkSqlWriter.write(sqlContext, SaveMode.Append, noReconciliationOpts, df5)
824-
}
818+
val fifthBatchActualSchema = fetchActualSchema()
819+
val fifthBatchExpectedSchema = {
820+
val (structName, nameSpace) = AvroConversionUtils.getAvroRecordNameAndNamespace(hoodieFooTableName)
821+
AvroConversionUtils.convertStructTypeToAvroSchema(df5.schema, structName, nameSpace)
825822
}
823+
assertEquals(fifthBatchExpectedSchema, fifthBatchActualSchema)
826824
}
827825

828826
/**
@@ -1419,7 +1417,6 @@ object TestHoodieSparkSqlWriter {
14191417

14201418
def deletePartitionsWildcardTestParams(): java.util.stream.Stream[Arguments] = {
14211419
java.util.stream.Stream.of(
1422-
arguments("2015/03/*", Seq("2016/03/15")),
14231420
arguments("*5/03/1*", Seq("2016/03/15")),
14241421
arguments("2016/03/*", Seq("2015/03/16", "2015/03/17")))
14251422
}

hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestCOWDataSource.scala

+13-10
Original file line numberDiff line numberDiff line change
@@ -658,7 +658,7 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
658658
val countDownLatch = new CountDownLatch(2)
659659
for (x <- 1 to 2) {
660660
val thread = new Thread(new UpdateThread(dataGen, spark, commonOpts, basePath, x + "00", countDownLatch, numRetries))
661-
thread.setName((x + "00_THREAD").toString())
661+
thread.setName(x + "00_THREAD")
662662
thread.start()
663663
}
664664
countDownLatch.await(1, TimeUnit.MINUTES)
@@ -682,15 +682,18 @@ class TestCOWDataSource extends HoodieSparkClientTestBase with ScalaAssertionSup
682682
val insertRecs = recordsToStrings(dataGen.generateInserts(instantTime, 1000)).toList
683683
val updateDf = spark.read.json(spark.sparkContext.parallelize(updateRecs, 2))
684684
val insertDf = spark.read.json(spark.sparkContext.parallelize(insertRecs, 2))
685-
updateDf.union(insertDf).write.format("org.apache.hudi")
686-
.options(commonOpts)
687-
.option("hoodie.write.concurrency.mode", "optimistic_concurrency_control")
688-
.option("hoodie.cleaner.policy.failed.writes", "LAZY")
689-
.option("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider")
690-
.option(HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.key(), numRetries.toString)
691-
.mode(SaveMode.Append)
692-
.save(basePath)
693-
countDownLatch.countDown()
685+
try {
686+
updateDf.union(insertDf).write.format("org.apache.hudi")
687+
.options(commonOpts)
688+
.option("hoodie.write.concurrency.mode", "optimistic_concurrency_control")
689+
.option("hoodie.cleaner.policy.failed.writes", "LAZY")
690+
.option("hoodie.write.lock.provider", "org.apache.hudi.client.transaction.lock.InProcessLockProvider")
691+
.option(HoodieWriteConfig.NUM_RETRIES_ON_CONFLICT_FAILURES.key(), numRetries.toString)
692+
.mode(SaveMode.Append)
693+
.save(basePath)
694+
} finally {
695+
countDownLatch.countDown()
696+
}
694697
}
695698
}
696699

hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestDataSourceForBootstrap.scala

+14-21
Original file line numberDiff line numberDiff line change
@@ -171,8 +171,8 @@ class TestDataSourceForBootstrap {
171171
@CsvSource(value = Array(
172172
"org.apache.hudi.client.bootstrap.selector.MetadataOnlyBootstrapModeSelector,AVRO",
173173
// TODO(HUDI-5807) enable for spark native records
174-
/* "org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector,SPARK", */
175-
"org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector,AVRO",
174+
/* "org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector,SPARK",
175+
"org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector,AVRO",*/
176176
"org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector,SPARK"
177177
))
178178
def testMetadataBootstrapCOWHiveStylePartitioned(bootstrapSelector: String, recordType: HoodieRecordType): Unit = {
@@ -252,11 +252,8 @@ class TestDataSourceForBootstrap {
252252
verifyIncrementalViewResult(commitInstantTime1, commitInstantTime2, isPartitioned = true, isHiveStylePartitioned = true)
253253
}
254254

255-
@ParameterizedTest
256-
@EnumSource(value = classOf[HoodieRecordType],
257-
// TODO(HUDI-5807) enable for spark native records
258-
names = Array("AVRO" /*, "SPARK" */))
259-
def testMetadataBootstrapCOWPartitioned(recordType: HoodieRecordType): Unit = {
255+
@Test
256+
def testMetadataBootstrapCOWPartitioned(): Unit = {
260257
val timestamp = Instant.now.toEpochMilli
261258
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
262259

@@ -268,7 +265,7 @@ class TestDataSourceForBootstrap {
268265
.mode(SaveMode.Overwrite)
269266
.save(srcPath)
270267

271-
val writeOpts = commonOpts ++ getRecordTypeOpts(recordType) ++ Map(
268+
val writeOpts = commonOpts ++ getRecordTypeOpts(HoodieRecordType.AVRO) ++ Map(
272269
DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true",
273270
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "datestr"
274271
)
@@ -331,9 +328,8 @@ class TestDataSourceForBootstrap {
331328
verifyIncrementalViewResult(commitInstantTime1, commitInstantTime3, isPartitioned = true, isHiveStylePartitioned = true)
332329
}
333330

334-
@ParameterizedTest
335-
@ValueSource(booleans = Array(true, false))
336-
def testMetadataBootstrapMORPartitionedInlineClustering(enableRowWriter: Boolean): Unit = {
331+
@Test
332+
def testMetadataBootstrapMORPartitionedInlineClustering(): Unit = {
337333
val timestamp = Instant.now.toEpochMilli
338334
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
339335
// Prepare source data
@@ -343,7 +339,7 @@ class TestDataSourceForBootstrap {
343339
.mode(SaveMode.Overwrite)
344340
.save(srcPath)
345341

346-
val writeOpts = commonOpts ++ getRecordTypeOpts(HoodieRecordType.AVRO) ++ Map(
342+
val writeOpts = commonOpts ++ Map(
347343
DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true",
348344
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "datestr"
349345
)
@@ -370,7 +366,6 @@ class TestDataSourceForBootstrap {
370366
.options(writeOpts)
371367
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
372368
.option(DataSourceWriteOptions.TABLE_TYPE.key, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL)
373-
.option(DataSourceWriteOptions.ENABLE_ROW_WRITER.key, enableRowWriter.toString)
374369
.option(HoodieClusteringConfig.INLINE_CLUSTERING.key, "true")
375370
.option(HoodieClusteringConfig.INLINE_CLUSTERING_MAX_COMMITS.key, "1")
376371
.option(HoodieClusteringConfig.PLAN_STRATEGY_SORT_COLUMNS.key, "datestr")
@@ -464,9 +459,8 @@ class TestDataSourceForBootstrap {
464459
assertEquals(numRecordsUpdate, hoodieROViewDFWithBasePath.filter(s"timestamp == $updateTimestamp").count())
465460
}
466461

467-
@ParameterizedTest
468-
@EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK"))
469-
def testMetadataBootstrapMORPartitioned(recordType: HoodieRecordType): Unit = {
462+
@Test
463+
def testMetadataBootstrapMORPartitioned(): Unit = {
470464
val timestamp = Instant.now.toEpochMilli
471465
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
472466

@@ -478,7 +472,7 @@ class TestDataSourceForBootstrap {
478472
.mode(SaveMode.Overwrite)
479473
.save(srcPath)
480474

481-
val writeOpts = commonOpts ++ getRecordTypeOpts(recordType) ++ Map(
475+
val writeOpts = commonOpts ++ Map(
482476
DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true",
483477
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "datestr"
484478
)
@@ -550,9 +544,8 @@ class TestDataSourceForBootstrap {
550544
assertEquals(0, hoodieROViewDF3.filter(s"timestamp == $updateTimestamp").count())
551545
}
552546

553-
@ParameterizedTest
554-
@EnumSource(value = classOf[HoodieRecordType], names = Array("AVRO", "SPARK"))
555-
def testFullBootstrapCOWPartitioned(recordType: HoodieRecordType): Unit = {
547+
@Test
548+
def testFullBootstrapCOWPartitioned(): Unit = {
556549
val timestamp = Instant.now.toEpochMilli
557550
val jsc = JavaSparkContext.fromSparkContext(spark.sparkContext)
558551

@@ -564,7 +557,7 @@ class TestDataSourceForBootstrap {
564557
.mode(SaveMode.Overwrite)
565558
.save(srcPath)
566559

567-
val writeOpts = commonOpts ++ getRecordTypeOpts(recordType) ++ Map(
560+
val writeOpts = commonOpts ++ Map(
568561
DataSourceWriteOptions.HIVE_STYLE_PARTITIONING.key -> "true",
569562
DataSourceWriteOptions.PARTITIONPATH_FIELD.key -> "datestr"
570563
)

hudi-spark-datasource/hudi-spark/src/test/scala/org/apache/hudi/functional/TestSparkDataSource.scala

+24-56
Original file line numberDiff line numberDiff line change
@@ -55,26 +55,16 @@ class TestSparkDataSource extends SparkClientFunctionalTestHarness {
5555

5656
@ParameterizedTest
5757
@CsvSource(value = Array(
58-
"COPY_ON_WRITE|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
59-
"COPY_ON_WRITE|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
60-
"COPY_ON_WRITE|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
61-
"COPY_ON_WRITE|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
62-
"COPY_ON_WRITE|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
63-
"COPY_ON_WRITE|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
64-
"COPY_ON_WRITE|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
65-
"COPY_ON_WRITE|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
66-
"COPY_ON_WRITE|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
67-
"MERGE_ON_READ|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
68-
"MERGE_ON_READ|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
69-
"MERGE_ON_READ|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
70-
"MERGE_ON_READ|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
71-
"MERGE_ON_READ|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
72-
"MERGE_ON_READ|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
73-
"MERGE_ON_READ|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
74-
"MERGE_ON_READ|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
75-
"MERGE_ON_READ|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM"
58+
"COPY_ON_WRITE|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
59+
"COPY_ON_WRITE|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
60+
"COPY_ON_WRITE|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
61+
"MERGE_ON_READ|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
62+
"MERGE_ON_READ|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
63+
"MERGE_ON_READ|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM"
7664
), delimiter = '|')
77-
def testCoreFlow(tableType: String, isMetadataEnabledOnWrite: Boolean, isMetadataEnabledOnRead: Boolean, keyGenClass: String, indexType: String): Unit = {
65+
def testCoreFlow(tableType: String, keyGenClass: String, indexType: String): Unit = {
66+
val isMetadataEnabledOnWrite = true
67+
val isMetadataEnabledOnRead = true
7868
val partitionField = if (classOf[NonpartitionedKeyGenerator].getName.equals(keyGenClass)) "" else "partition"
7969
val options: Map[String, String] = commonOpts +
8070
(HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabledOnWrite)) +
@@ -220,44 +210,22 @@ class TestSparkDataSource extends SparkClientFunctionalTestHarness {
220210

221211
@ParameterizedTest
222212
@CsvSource(value = Array(
223-
"COPY_ON_WRITE|insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
224-
"COPY_ON_WRITE|insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
225-
"COPY_ON_WRITE|insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
226-
"COPY_ON_WRITE|insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
227-
"COPY_ON_WRITE|insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
228-
"COPY_ON_WRITE|insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
229-
"COPY_ON_WRITE|insert|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
230-
"COPY_ON_WRITE|insert|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
231-
"COPY_ON_WRITE|insert|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
232-
"MERGE_ON_READ|insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
233-
"MERGE_ON_READ|insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
234-
"MERGE_ON_READ|insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
235-
"MERGE_ON_READ|insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
236-
"MERGE_ON_READ|insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
237-
"MERGE_ON_READ|insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
238-
"MERGE_ON_READ|insert|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
239-
"MERGE_ON_READ|insert|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
240-
"MERGE_ON_READ|insert|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
241-
"COPY_ON_WRITE|bulk_insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
242-
"COPY_ON_WRITE|bulk_insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
243-
"COPY_ON_WRITE|bulk_insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
244-
"COPY_ON_WRITE|bulk_insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
245-
"COPY_ON_WRITE|bulk_insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
246-
"COPY_ON_WRITE|bulk_insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
247-
"COPY_ON_WRITE|bulk_insert|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
248-
"COPY_ON_WRITE|bulk_insert|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
249-
"COPY_ON_WRITE|bulk_insert|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
250-
"MERGE_ON_READ|bulk_insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
251-
"MERGE_ON_READ|bulk_insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
252-
"MERGE_ON_READ|bulk_insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
253-
"MERGE_ON_READ|bulk_insert|false|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
254-
"MERGE_ON_READ|bulk_insert|true|false|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
255-
"MERGE_ON_READ|bulk_insert|true|true|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
256-
"MERGE_ON_READ|bulk_insert|false|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
257-
"MERGE_ON_READ|bulk_insert|true|false|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
258-
"MERGE_ON_READ|bulk_insert|true|true|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM"
213+
"COPY_ON_WRITE|insert|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
214+
"COPY_ON_WRITE|insert|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
215+
"COPY_ON_WRITE|insert|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
216+
"MERGE_ON_READ|insert|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
217+
"MERGE_ON_READ|insert|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
218+
"MERGE_ON_READ|insert|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
219+
"COPY_ON_WRITE|bulk_insert|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
220+
"COPY_ON_WRITE|bulk_insert|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
221+
"COPY_ON_WRITE|bulk_insert|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM",
222+
"MERGE_ON_READ|bulk_insert|org.apache.hudi.keygen.SimpleKeyGenerator|BLOOM",
223+
"MERGE_ON_READ|bulk_insert|org.apache.hudi.keygen.SimpleKeyGenerator|SIMPLE",
224+
"MERGE_ON_READ|bulk_insert|org.apache.hudi.keygen.NonpartitionedKeyGenerator|GLOBAL_BLOOM"
259225
), delimiter = '|')
260-
def testImmutableUserFlow(tableType: String, operation: String, isMetadataEnabledOnWrite: Boolean, isMetadataEnabledOnRead: Boolean, keyGenClass: String, indexType: String): Unit = {
226+
def testImmutableUserFlow(tableType: String, operation: String, keyGenClass: String, indexType: String): Unit = {
227+
val isMetadataEnabledOnWrite = true
228+
val isMetadataEnabledOnRead = true
261229
val partitionField = if (classOf[NonpartitionedKeyGenerator].getName.equals(keyGenClass)) "" else "partition"
262230
val options: Map[String, String] = commonOpts +
263231
(HoodieMetadataConfig.ENABLE.key -> String.valueOf(isMetadataEnabledOnWrite)) +

0 commit comments

Comments
 (0)