1
0

[HUDI-1427] Fix FileAlreadyExistsException when set HOODIE_AUTO_COMMIT_PROP to true (#2295)

This commit is contained in:
pengzhiwei
2020-12-05 08:07:25 +08:00
committed by GitHub
parent 62b392b49c
commit 319b7a58e4
2 changed files with 16 additions and 2 deletions

View File

@@ -165,7 +165,7 @@ private[hudi] object HoodieSparkSqlWriter {
// Create a HoodieWriteClient & issue the write.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc, schema.toString, path.get,
tblName, mapAsJavaMap(parameters)
tblName, mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)
)).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
@@ -205,7 +205,8 @@ private[hudi] object HoodieSparkSqlWriter {
// Create a HoodieWriteClient & issue the delete.
val client = hoodieWriteClient.getOrElse(DataSourceUtils.createHoodieClient(jsc,
Schema.create(Schema.Type.NULL).toString, path.get, tblName,
mapAsJavaMap(parameters))).asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
mapAsJavaMap(parameters - HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP)))
.asInstanceOf[SparkRDDWriteClient[HoodieRecordPayload[Nothing]]]
if (isAsyncCompactionEnabled(client, tableConfig, parameters, jsc.hadoopConfiguration())) {
asyncCompactionTriggerFn.get.apply(client)

View File

@@ -307,4 +307,17 @@ class TestCOWDataSource extends HoodieClientTestBase {
}
})
}
@Test def testWithAutoCommitOn(): Unit = {
val records1 = recordsToStrings(dataGen.generateInserts("000", 100)).toList
val inputDF1 = spark.read.json(spark.sparkContext.parallelize(records1, 2))
inputDF1.write.format("org.apache.hudi")
.options(commonOpts)
.option(DataSourceWriteOptions.OPERATION_OPT_KEY, DataSourceWriteOptions.INSERT_OPERATION_OPT_VAL)
.option(HoodieWriteConfig.HOODIE_AUTO_COMMIT_PROP, "true")
.mode(SaveMode.Overwrite)
.save(basePath)
assertTrue(HoodieDataSourceHelpers.hasNewCommits(fs, basePath, "000"))
}
}