[
https://issues.apache.org/jira/browse/HUDI-4121?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17539158#comment-17539158
]
Yong Zhang commented on HUDI-4121:
----------------------------------
I found in the SparkRDDWriteClient, it implements preCommit
[https://github.com/apache/hudi/blob/551aa959c57721a5cc4d3f63f79e0201978980a2/hudi-client/hudi-spark-client/src/main/java/org/apache/hudi/client/SparkRDDWriteClient.java#L471.]
But the java client doesn't do similar things
> The java client missing some supports on the conflict handling?
> ---------------------------------------------------------------
>
> Key: HUDI-4121
> URL: https://issues.apache.org/jira/browse/HUDI-4121
> Project: Apache Hudi
> Issue Type: Bug
> Reporter: Yong Zhang
> Priority: Major
>
> When I enable the concurrency in the hudi java writer, it looks like
> something is wrong when committing at the same time.
>
> The exception:
>
> ```
>
> {{org.apache.hudi.exception.HoodieIOException: Failed to create file
> file:/tmp/integration/hudi/.hoodie/20220517094051766.commit
> at
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createImmutableFileInPath(HoodieActiveTimeline.java:745)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:560)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.transitionState(HoodieActiveTimeline.java:536)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.saveAsComplete(HoodieActiveTimeline.java:183)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:270)
> ~[hudi-client-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.client.BaseHoodieWriteClient.commitStats(BaseHoodieWriteClient.java:234)
> ~[hudi-client-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.client.HoodieJavaWriteClient.commit(HoodieJavaWriteClient.java:88)
> ~[hudi-java-client-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.client.HoodieJavaWriteClient.commit(HoodieJavaWriteClient.java:51)
> ~[hudi-java-client-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.client.BaseHoodieWriteClient.commit(BaseHoodieWriteClient.java:206)
> ~[hudi-client-common-0.11.0.jar:0.11.0]
> at
> org.apache.pulsar.ecosystem.io.sink.hudi.BufferedConnectWriter.flushRecords(BufferedConnectWriter.java:82)
> ~[PqY5lYEJSWPWMDq7E5HC2Q/:?]
> at
> org.apache.pulsar.ecosystem.io.sink.hudi.HoodieWriter.flush(HoodieWriter.java:85)
> ~[PqY5lYEJSWPWMDq7E5HC2Q/:?]
> at
> org.apache.pulsar.ecosystem.io.sink.SinkWriter.commitIfNeed(SinkWriter.java:128)
> ~[PqY5lYEJSWPWMDq7E5HC2Q/:?]
> at
> org.apache.pulsar.ecosystem.io.sink.SinkWriter.run(SinkWriter.java:113)
> [PqY5lYEJSWPWMDq7E5HC2Q/:?]
> at
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
> [?:1.8.0_201]
> at
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
> [?:1.8.0_201]
> at
> io.netty.util.concurrent.FastThreadLocalRunnable.run(FastThreadLocalRunnable.java:30)
> [netty-common-4.1.77.Final.jar:4.1.77.Final]
> at java.lang.Thread.run(Thread.java:748) [?:1.8.0_201]
> Caused by: org.apache.hadoop.fs.FileAlreadyExistsException: File already
> exists: file:/tmp/integration/hudi/.hoodie/20220517094051766.commit
> at
> org.apache.hadoop.fs.RawLocalFileSystem.create(RawLocalFileSystem.java:315)
> ~[hadoop-common-3.2.2.jar:?]
> at
> org.apache.hadoop.fs.RawLocalFileSystem.create(RawLocalFileSystem.java:353)
> ~[hadoop-common-3.2.2.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem$ChecksumFSOutputSummer.<init>(ChecksumFileSystem.java:403)
> ~[hadoop-common-3.2.2.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:466)
> ~[hadoop-common-3.2.2.jar:?]
> at
> org.apache.hadoop.fs.ChecksumFileSystem.create(ChecksumFileSystem.java:445)
> ~[hadoop-common-3.2.2.jar:?]
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1125)
> ~[hadoop-common-3.2.2.jar:?]
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:1105)
> ~[hadoop-common-3.2.2.jar:?]
> at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:994)
> ~[hadoop-common-3.2.2.jar:?]
> at
> org.apache.hudi.common.fs.HoodieWrapperFileSystem.lambda$create$2(HoodieWrapperFileSystem.java:222)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.fs.HoodieWrapperFileSystem.executeFuncWithTimeMetrics(HoodieWrapperFileSystem.java:101)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.fs.HoodieWrapperFileSystem.create(HoodieWrapperFileSystem.java:221)
> ~[hudi-common-0.11.0.jar:0.11.0]
> at
> org.apache.hudi.common.table.timeline.HoodieActiveTimeline.createImmutableFileInPath(HoodieActiveTimeline.java:740)
> ~[hudi-common-0.11.0.jar:0.11.0]
> ... 16 more}}
> ```
>
> And my hudi writer configuration:
> ```
>
> {{"hoodie.table.name": "hudi-connector-test",
> "hoodie.table.type": "COPY_ON_WRITE",
> "hoodie.base.path": "file:///tmp/integration/hudi",
> "hoodie.clean.async": "true",
> "hoodie.write.concurrency.mode": "optimistic_concurrency_control",
> "hoodie.cleaner.policy.failed.writes": "LAZY",
> "hoodie.write.lock.provider":
> "org.apache.hudi.client.transaction.lock.ZookeeperBasedLockProvider",
> "hoodie.write.lock.zookeeper.url": "localhost",
> "hoodie.write.lock.zookeeper.port": "2181",
> "hoodie.write.lock.zookeeper.lock_key": "pulsar_hudi",
> "hoodie.write.lock.zookeeper.base_path": "/hudi",
> "hoodie.datasource.write.recordkey.field": "id",
> "hoodie.datasource.write.partitionpath.field": "id",}}
> ```
--
This message was sent by Atlassian Jira
(v8.20.7#820007)