@@ -23,17 +23,17 @@ import org.apache.hudi.client.bootstrap.selector.FullRecordBootstrapModeSelector
2323import org .apache .hudi .common .fs .FSUtils
2424import org .apache .hudi .common .table .timeline .HoodieTimeline
2525import org .apache .hudi .config .{HoodieBootstrapConfig , HoodieCompactionConfig , HoodieWriteConfig }
26- import org .apache .hudi .keygen .SimpleKeyGenerator
26+ import org .apache .hudi .keygen .{ NonpartitionedKeyGenerator , SimpleKeyGenerator }
2727import org .apache .hudi .{DataSourceReadOptions , DataSourceWriteOptions , HoodieDataSourceHelpers }
2828import org .apache .spark .api .java .JavaSparkContext
2929import org .apache .spark .sql .functions .{col , lit }
3030import org .apache .spark .sql .{SaveMode , SparkSession }
3131import org .junit .jupiter .api .Assertions .assertEquals
3232import org .junit .jupiter .api .io .TempDir
3333import org .junit .jupiter .api .{AfterEach , BeforeEach , Test }
34+
3435import java .time .Instant
3536import java .util .Collections
36-
3737import scala .collection .JavaConverters ._
3838
3939class TestDataSourceForBootstrap {
@@ -102,9 +102,12 @@ class TestDataSourceForBootstrap {
102102 .save(srcPath)
103103
104104 // Perform bootstrap
105+ val bootstrapKeygenClass = classOf [NonpartitionedKeyGenerator ].getName
106+ val options = commonOpts.- (DataSourceWriteOptions .PARTITIONPATH_FIELD .key)
105107 val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
106108 DataSourceWriteOptions .COW_TABLE_TYPE_OPT_VAL ,
107- extraOpts = Map (DataSourceWriteOptions .KEYGENERATOR_CLASS_NAME .key -> " org.apache.hudi.keygen.NonpartitionedKeyGenerator" )
109+ extraOpts = options ++ Map (DataSourceWriteOptions .KEYGENERATOR_CLASS_NAME .key -> bootstrapKeygenClass),
110+ bootstrapKeygenClass = bootstrapKeygenClass
108111 )
109112 // check marked directory clean up
110113 assert(! fs.exists(new Path (basePath, " .hoodie/.temp/00000000000001" )))
@@ -123,10 +126,10 @@ class TestDataSourceForBootstrap {
123126
124127 updateDF.write
125128 .format(" hudi" )
126- .options(commonOpts )
129+ .options(options )
127130 .option(DataSourceWriteOptions .OPERATION .key, DataSourceWriteOptions .UPSERT_OPERATION_OPT_VAL )
128131 .option(DataSourceWriteOptions .TABLE_TYPE .key, DataSourceWriteOptions .COW_TABLE_TYPE_OPT_VAL )
129- .option(DataSourceWriteOptions .KEYGENERATOR_CLASS_NAME .key, " org.apache.hudi.keygen.NonpartitionedKeyGenerator " )
132+ .option(DataSourceWriteOptions .KEYGENERATOR_CLASS_NAME .key, bootstrapKeygenClass )
130133 .mode(SaveMode .Append )
131134 .save(basePath)
132135
@@ -163,8 +166,8 @@ class TestDataSourceForBootstrap {
163166 // Perform bootstrap
164167 val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
165168 DataSourceWriteOptions .COW_TABLE_TYPE_OPT_VAL ,
166- Some ( " datestr" ),
167- Map ( DataSourceWriteOptions . HIVE_STYLE_PARTITIONING .key -> " true " ) )
169+ commonOpts.updated( DataSourceWriteOptions . PARTITIONPATH_FIELD .key, " datestr" ) ++ Map ( DataSourceWriteOptions . HIVE_STYLE_PARTITIONING .key -> " true " ),
170+ classOf [ SimpleKeyGenerator ].getName )
168171
169172 // check marked directory clean up
170173 assert(! fs.exists(new Path (basePath, " .hoodie/.temp/00000000000001" )))
@@ -227,7 +230,9 @@ class TestDataSourceForBootstrap {
227230
228231 // Perform bootstrap
229232 val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
230- DataSourceWriteOptions .COW_TABLE_TYPE_OPT_VAL , Some (" datestr" ))
233+ DataSourceWriteOptions .COW_TABLE_TYPE_OPT_VAL ,
234+ commonOpts.updated(DataSourceWriteOptions .PARTITIONPATH_FIELD .key, " datestr" ),
235+ classOf [SimpleKeyGenerator ].getName)
231236
232237 // Read bootstrapped table and verify count using glob path
233238 val hoodieROViewDF1 = spark.read.format(" hudi" ).load(basePath + " /*" )
@@ -302,7 +307,9 @@ class TestDataSourceForBootstrap {
302307
303308 // Perform bootstrap
304309 val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
305- DataSourceWriteOptions .MOR_TABLE_TYPE_OPT_VAL , Some (" datestr" ))
310+ DataSourceWriteOptions .MOR_TABLE_TYPE_OPT_VAL ,
311+ commonOpts.updated(DataSourceWriteOptions .PARTITIONPATH_FIELD .key, " datestr" ),
312+ classOf [SimpleKeyGenerator ].getName)
306313
307314 // Read bootstrapped table and verify count
308315 val hoodieROViewDF1 = spark.read.format(" hudi" )
@@ -367,7 +374,9 @@ class TestDataSourceForBootstrap {
367374
368375 // Perform bootstrap
369376 val commitInstantTime1 = runMetadataBootstrapAndVerifyCommit(
370- DataSourceWriteOptions .MOR_TABLE_TYPE_OPT_VAL , Some (" datestr" ))
377+ DataSourceWriteOptions .MOR_TABLE_TYPE_OPT_VAL ,
378+ commonOpts.updated(DataSourceWriteOptions .PARTITIONPATH_FIELD .key, " datestr" ),
379+ classOf [SimpleKeyGenerator ].getName)
371380
372381 // Read bootstrapped table and verify count
373382 val hoodieROViewDF1 = spark.read.format(" hudi" )
@@ -497,18 +506,16 @@ class TestDataSourceForBootstrap {
497506 }
498507
499508 def runMetadataBootstrapAndVerifyCommit (tableType : String ,
500- partitionColumns : Option [String ] = None ,
501- extraOpts : Map [ String , String ] = Map .empty ): String = {
509+ extraOpts : Map [String , String ] = Map .empty ,
510+ bootstrapKeygenClass : String ): String = {
502511 val bootstrapDF = spark.emptyDataFrame
503512 bootstrapDF.write
504513 .format(" hudi" )
505- .options(commonOpts)
506514 .options(extraOpts)
507515 .option(DataSourceWriteOptions .OPERATION .key, DataSourceWriteOptions .BOOTSTRAP_OPERATION_OPT_VAL )
508516 .option(DataSourceWriteOptions .TABLE_TYPE .key, tableType)
509- .option(DataSourceWriteOptions .PARTITIONPATH_FIELD .key, partitionColumns.getOrElse(" " ))
510517 .option(HoodieBootstrapConfig .BASE_PATH .key, srcPath)
511- .option(HoodieBootstrapConfig .KEYGEN_CLASS_NAME .key, classOf [ SimpleKeyGenerator ].getName )
518+ .option(HoodieBootstrapConfig .KEYGEN_CLASS_NAME .key, bootstrapKeygenClass )
512519 .mode(SaveMode .Overwrite )
513520 .save(basePath)
514521
@@ -528,7 +535,7 @@ class TestDataSourceForBootstrap {
528535 .load(basePath)
529536
530537 assertEquals(numRecords, hoodieIncViewDF1.count())
531- var countsPerCommit = hoodieIncViewDF1.groupBy(" _hoodie_commit_time" ).count().collect();
538+ var countsPerCommit = hoodieIncViewDF1.groupBy(" _hoodie_commit_time" ).count().collect()
532539 assertEquals(1 , countsPerCommit.length)
533540 assertEquals(bootstrapCommitInstantTime, countsPerCommit(0 ).get(0 ))
534541
@@ -537,10 +544,10 @@ class TestDataSourceForBootstrap {
537544 val hoodieIncViewDF2 = spark.read.format(" hudi" )
538545 .option(DataSourceReadOptions .QUERY_TYPE .key, DataSourceReadOptions .QUERY_TYPE_INCREMENTAL_OPT_VAL )
539546 .option(DataSourceReadOptions .BEGIN_INSTANTTIME .key, bootstrapCommitInstantTime)
540- .load(basePath);
547+ .load(basePath)
541548
542549 assertEquals(numRecordsUpdate, hoodieIncViewDF2.count())
543- countsPerCommit = hoodieIncViewDF2.groupBy(" _hoodie_commit_time" ).count().collect();
550+ countsPerCommit = hoodieIncViewDF2.groupBy(" _hoodie_commit_time" ).count().collect()
544551 assertEquals(1 , countsPerCommit.length)
545552 assertEquals(latestCommitInstantTime, countsPerCommit(0 ).get(0 ))
546553
0 commit comments