You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hudi.apache.org by GitBox <gi...@apache.org> on 2022/04/15 09:16:27 UTC

[GitHub] [hudi] Guanpx opened a new issue, #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Guanpx opened a new issue, #5330:
URL: https://github.com/apache/hudi/issues/5330

   **Describe the problem you faced**
   
   use flink1.13 ,bucket index , cow ,hudi-0.11.0(not latest) 
   
   **To Reproduce**
   
   Steps to reproduce the behavior:
   
   1. start flink job
   2. cancel flink job
   3. repeat 1-2 some times
   4. start job,then that Exception was occured 
   
   
   **Environment Description**
   
   * Hudi version : 0.11.0
   
   * Flink version : 1.13.2
   
   * Hadoop version : 3.0.0
   
   * Storage (HDFS/S3/GCS..) :HDFS
   
   * Running on Docker? (yes/no) : no
   
   
   **Additional context**
    
   ![image](https://user-images.githubusercontent.com/29246713/163552259-4e5f0215-e696-4b2a-a11c-4b555a2aa220.png)
   
   
   **Stacktrace**
   
   ```
   java.lang.RuntimeException: Duplicate fileID 00000007-????-????-????-40bee2bd5a70 from bucket 7 of partition  found during the BucketStreamWriteFunction index bootstrap.
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.lambda$bootstrapIndexIfNeed$1(BucketStreamWriteFunction.java:179)
   	at java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:184)
   	at java.util.stream.ReferencePipeline$3$1.accept(ReferencePipeline.java:193)
   	at java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1382)
   	at java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:481)
   	at java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:471)
   	at java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:151)
   	at java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:174)
   	at java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
   	at java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:418)
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.bootstrapIndexIfNeed(BucketStreamWriteFunction.java:173)
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.processElement(BucketStreamWriteFunction.java:123)
   	at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66)
   	at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:205)
   	at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:134)
   	at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:105)
   	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:66)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:423)
   	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:204)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:681)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.executeInvoke(StreamTask.java:636)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.runWithCleanUpOnFail(StreamTask.java:647)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:620)
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:779)
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:566)
   	at java.lang.Thread.run(Thread.java:748)
   
   ```
   
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wxplovecc commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
wxplovecc commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1100693250

   see https://github.com/apache/hudi/pull/5185


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] jarrodcodes commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by "jarrodcodes (via GitHub)" <gi...@apache.org>.
jarrodcodes commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1557410683

   > Did you use the COW table or MOR?
   
   We are using COW.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1229548003

   closing the issue since the linked PR is landed. feel free to reopen or open a new issue if you are still facing issues.
   thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] Guanpx commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
Guanpx commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1100019832

   cc @danny0405 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan closed issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
nsivabalan closed issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition  found during the BucketStreamWriteFunction index bootstrap.
URL: https://github.com/apache/hudi/issues/5330


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] LinMingQiang commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
LinMingQiang commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1223475113

   see https://github.com/apache/hudi/pull/5763


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1101059248

   Thanks for the PR @wxplovecc , can you explain why the #5185 patch fixed the bug ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1556374428

   Did you write to an exiting table or a fresh new table then?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] yihua commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
yihua commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1111750027

   @Guanpx have you tried the latest master and see if the fix solves the problem for you?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by "danny0405 (via GitHub)" <gi...@apache.org>.
danny0405 commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1557391885

   Did you use the COW table or MOR?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] jarrodcodes commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by "jarrodcodes (via GitHub)" <gi...@apache.org>.
jarrodcodes commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1557241390

   > Did you write to an exiting table or a fresh new table then?
   
   This was a fresh table. Flink had restarted a few times due to a memory issue. It looks like this occured after that.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] danny0405 commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
danny0405 commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1101255556

   @Guanpx  Did you use the code of latest master ? The shuffle strategy in https://github.com/apache/hudi/blob/b8e465fdfcac1961fe05ed44993c8c6139e13b31/hudi-flink-datasource/hudi-flink/src/main/java/org/apache/hudi/sink/partitioner/BucketIndexPartitioner.java#L44 determines that one bucket number is always handled by a single task, that means no two write tasks would write to the same bucket number, and within one write task, there is {bucket id} to {file id} mapping.
   
   Do you guys have any idea how the error happens ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] Guanpx commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
Guanpx commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1101923406

   > 
   I will try latest master tomorrow and debug that code ~


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] nsivabalan commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
nsivabalan commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1126595835

   @Guanpx : do you have any updates on this issue. if latest master works, feel free to close out the github issue. 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] wxplovecc commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by GitBox <gi...@apache.org>.
wxplovecc commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1101965808

   > Thanks for the PR @wxplovecc , can you explain why the #5185 patch fixed the bug ?
   
   The mor table rollback was not delete the log files , If the job failed before first successful commit it will left some invalid 
   log files, the PR #5185 was reuse the left invalid fileId by scan the left log file when restart the job 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [hudi] jarrodcodes commented on issue #5330: [SUPPORT] [BUG] Duplicate fileID ??? from bucket ?? of partition found during the BucketStreamWriteFunction index bootstrap.

Posted by "jarrodcodes (via GitHub)" <gi...@apache.org>.
jarrodcodes commented on issue #5330:
URL: https://github.com/apache/hudi/issues/5330#issuecomment-1555951277

   Seeing this issue with Flink 1.15 and Hudi 0.12.3: 
   
   `java.lang.RuntimeException: Duplicate fileId 00000002-fb1c-47ac-a203-397ffbbd9b91 from bucket 2 of partition dt=2022-11-21 found during the BucketStreamWriteFunction index bootstrap.
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.lambda$bootstrapIndexIfNeed$1(BucketStreamWriteFunction.java:162)
   	at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(Unknown Source)
   	at java.base/java.util.stream.ReferencePipeline$3$1.accept(Unknown Source)
   	at java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(Unknown Source)
   	at java.base/java.util.stream.AbstractPipeline.copyInto(Unknown Source)
   	at java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(Unknown Source)
   	at java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(Unknown Source)
   	at java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(Unknown Source)
   	at java.base/java.util.stream.AbstractPipeline.evaluate(Unknown Source)
   	at java.base/java.util.stream.ReferencePipeline.forEach(Unknown Source)
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.bootstrapIndexIfNeed(BucketStreamWriteFunction.java:155)
   	at org.apache.hudi.sink.bucket.BucketStreamWriteFunction.processElement(BucketStreamWriteFunction.java:111)
   	at org.apache.flink.streaming.api.operators.ProcessOperator.processElement(ProcessOperator.java:66)
   	at org.apache.flink.streaming.runtime.tasks.OneInputStreamTask$StreamTaskNetworkOutput.emitRecord(OneInputStreamTask.java:233)
   	at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.processElement(AbstractStreamTaskNetworkInput.java:134)
   	at org.apache.flink.streaming.runtime.io.AbstractStreamTaskNetworkInput.emitNext(AbstractStreamTaskNetworkInput.java:105)
   	at org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:519)
   	at org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:203)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:804)
   	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:753)
   	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:948)
   	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:927)
   	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:741)
   	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:563)
   	at java.base/java.lang.Thread.run(Unknown Source)
   `
   Looking for any suggestions, thank you.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@hudi.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org