|
17 | 17 |
|
18 | 18 | package org.apache.hudi.functional |
19 | 19 |
|
| 20 | +import java.time.Instant |
20 | 21 | import java.util |
21 | | -import java.util.{Date, UUID} |
| 22 | +import java.util.{Collections, Date, UUID} |
22 | 23 |
|
23 | 24 | import org.apache.commons.io.FileUtils |
24 | 25 | import org.apache.hudi.DataSourceWriteOptions._ |
25 | | -import org.apache.hudi.client.SparkRDDWriteClient |
| 26 | +import org.apache.hudi.client.{SparkRDDWriteClient, TestBootstrap} |
26 | 27 | import org.apache.hudi.common.model.{HoodieRecord, HoodieRecordPayload} |
27 | 28 | import org.apache.hudi.common.testutils.HoodieTestDataGenerator |
28 | | -import org.apache.hudi.config.HoodieWriteConfig |
| 29 | +import org.apache.hudi.config.{HoodieBootstrapConfig, HoodieWriteConfig} |
29 | 30 | import org.apache.hudi.exception.HoodieException |
30 | | -import org.apache.hudi.keygen.SimpleKeyGenerator |
| 31 | +import org.apache.hudi.keygen.{NonpartitionedKeyGenerator, SimpleKeyGenerator} |
31 | 32 | import org.apache.hudi.testutils.DataSourceTestUtils |
32 | 33 | import org.apache.hudi.{AvroConversionUtils, DataSourceUtils, DataSourceWriteOptions, HoodieSparkSqlWriter, HoodieWriterUtils} |
33 | 34 | import org.apache.spark.SparkContext |
@@ -341,6 +342,61 @@ class HoodieSparkSqlWriterSuite extends FunSuite with Matchers { |
341 | 342 | } |
342 | 343 | }) |
343 | 344 |
|
| 345 | + List(DataSourceWriteOptions.COW_TABLE_TYPE_OPT_VAL, DataSourceWriteOptions.MOR_TABLE_TYPE_OPT_VAL) |
| 346 | + .foreach(tableType => { |
| 347 | + test("test HoodieSparkSqlWriter functionality with datasource bootstrap for " + tableType) { |
| 348 | + initSparkContext("test_bootstrap_datasource") |
| 349 | + val path = java.nio.file.Files.createTempDirectory("hoodie_test_path") |
| 350 | + val srcPath = java.nio.file.Files.createTempDirectory("hoodie_bootstrap_source_path") |
| 351 | + |
| 352 | + try { |
| 353 | + |
| 354 | + val hoodieFooTableName = "hoodie_foo_tbl" |
| 355 | + |
| 356 | + val sourceDF = TestBootstrap.generateTestRawTripDataset(Instant.now.toEpochMilli, 0, 100, Collections.emptyList(), sc, |
| 357 | + spark.sqlContext) |
| 358 | + |
| 359 | + // Write source data non-partitioned |
| 360 | + sourceDF.write |
| 361 | + .format("parquet") |
| 362 | + .mode(SaveMode.Overwrite) |
| 363 | + .save(srcPath.toAbsolutePath.toString) |
| 364 | + |
| 365 | + val fooTableModifier = Map("path" -> path.toAbsolutePath.toString, |
| 366 | + HoodieBootstrapConfig.BOOTSTRAP_BASE_PATH_PROP -> srcPath.toAbsolutePath.toString, |
| 367 | + HoodieWriteConfig.TABLE_NAME -> hoodieFooTableName, |
| 368 | + DataSourceWriteOptions.TABLE_TYPE_OPT_KEY -> tableType, |
| 369 | + HoodieBootstrapConfig.BOOTSTRAP_PARALLELISM -> "4", |
| 370 | + DataSourceWriteOptions.OPERATION_OPT_KEY -> DataSourceWriteOptions.BOOTSTRAP_OPERATION_OPT_VAL, |
| 371 | + DataSourceWriteOptions.RECORDKEY_FIELD_OPT_KEY -> "_row_key", |
| 372 | + DataSourceWriteOptions.PARTITIONPATH_FIELD_OPT_KEY -> "partition", |
| 373 | + HoodieBootstrapConfig.BOOTSTRAP_KEYGEN_CLASS -> classOf[NonpartitionedKeyGenerator].getCanonicalName) |
| 374 | + val fooTableParams = HoodieWriterUtils.parametersWithWriteDefaults(fooTableModifier) |
| 375 | + |
| 376 | + val client = spy(DataSourceUtils.createHoodieClient( |
| 377 | + new JavaSparkContext(sc), |
| 378 | + null, |
| 379 | + path.toAbsolutePath.toString, |
| 380 | + hoodieFooTableName, |
| 381 | + mapAsJavaMap(fooTableParams)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]) |
| 382 | + |
| 383 | + HoodieSparkSqlWriter.bootstrap(sqlContext, SaveMode.Append, fooTableParams, spark.emptyDataFrame, Option.empty, |
| 384 | + Option(client)) |
| 385 | + |
| 386 | + // Verify that HoodieWriteClient is closed correctly |
| 387 | + verify(client, times(1)).close() |
| 388 | + |
| 389 | + // fetch all records from parquet files generated from write to hudi |
| 390 | + val actualDf = sqlContext.read.parquet(path.toAbsolutePath.toString) |
| 391 | + assert(actualDf.count == 100) |
| 392 | + } finally { |
| 393 | + spark.stop() |
| 394 | + FileUtils.deleteDirectory(path.toFile) |
| 395 | + FileUtils.deleteDirectory(srcPath.toFile) |
| 396 | + } |
| 397 | + } |
| 398 | + }) |
| 399 | + |
344 | 400 | case class Test(uuid: String, ts: Long) |
345 | 401 |
|
346 | 402 | import scala.collection.JavaConverters |
|
0 commit comments