[HUDI-2958] Automatically set spark.sql.parquet.writelegacyformat, when using bulkinsert to insert data which contains decimalType (#4253)
This commit is contained in:
@@ -723,4 +723,29 @@ class TestCOWDataSource extends HoodieClientTestBase {
|
||||
val result = spark.sql("select * from tmptable limit 1").collect()(0)
|
||||
result.schema.contains(new StructField("partition", StringType, true))
|
||||
}
|
||||
|
||||
@Test
|
||||
def testWriteSmallPrecisionDecimalTable(): Unit = {
|
||||
val records1 = recordsToStrings(dataGen.generateInserts("001", 5)).toList
|
||||
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
|
||||
.withColumn("shortDecimal", lit(new java.math.BigDecimal(s"2090.0000"))) // create decimalType(8, 4)
|
||||
inputDF1.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.BULK_INSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Overwrite)
|
||||
.save(basePath)
|
||||
|
||||
// update the value of shortDecimal
|
||||
val inputDF2 = inputDF1.withColumn("shortDecimal", lit(new java.math.BigDecimal(s"3090.0000")))
|
||||
inputDF2.write.format("org.apache.hudi")
|
||||
.options(commonOpts)
|
||||
.option(DataSourceWriteOptions.OPERATION.key, DataSourceWriteOptions.UPSERT_OPERATION_OPT_VAL)
|
||||
.mode(SaveMode.Append)
|
||||
.save(basePath)
|
||||
val readResult = spark.read.format("hudi").load(basePath)
|
||||
assert(readResult.count() == 5)
|
||||
// compare the test result
|
||||
assertEquals(inputDF2.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","),
|
||||
readResult.sort("_row_key").select("shortDecimal").collect().map(_.getDecimal(0).toPlainString).mkString(","))
|
||||
}
|
||||
}
|
||||
|
||||
Reference in New Issue
Block a user