Gitxu233 opened a new issue, #6955:
URL: https://github.com/apache/paimon/issues/6955

   ### Search before asking
   
   - [x] I searched in the [issues](https://github.com/apache/paimon/issues) 
and found nothing similar.
   
   
   ### Paimon version
   
   Paimon 1.2.0
   
   ### Compute Engine
   
   Flink 1.20.2
   JAVA 11
   
   ### Minimal reproduce step
   
   2026-01-05 19:28:18
   org.apache.flink.runtime.JobException: Recovery is suppressed by 
NoRestartBackoffTimeStrategy
        at 
org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.handleFailure(ExecutionFailureHandler.java:219)
        at 
org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.handleFailureAndReport(ExecutionFailureHandler.java:166)
        at 
org.apache.flink.runtime.executiongraph.failover.ExecutionFailureHandler.getFailureHandlingResult(ExecutionFailureHandler.java:121)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.recordTaskFailure(DefaultScheduler.java:281)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.handleTaskFailure(DefaultScheduler.java:272)
        at 
org.apache.flink.runtime.scheduler.adaptivebatch.AdaptiveBatchScheduler.handleTaskFailure(AdaptiveBatchScheduler.java:413)
        at 
org.apache.flink.runtime.scheduler.DefaultScheduler.onTaskFailed(DefaultScheduler.java:265)
        at 
org.apache.flink.runtime.scheduler.adaptivebatch.AdaptiveBatchScheduler.onTaskFailed(AdaptiveBatchScheduler.java:405)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.onTaskExecutionStateUpdate(SchedulerBase.java:788)
        at 
org.apache.flink.runtime.scheduler.SchedulerBase.updateTaskExecutionState(SchedulerBase.java:765)
        at 
org.apache.flink.runtime.scheduler.SchedulerNG.updateTaskExecutionState(SchedulerNG.java:83)
        at 
org.apache.flink.runtime.jobmaster.JobMaster.updateTaskExecutionState(JobMaster.java:515)
        at jdk.internal.reflect.GeneratedMethodAccessor913.invoke(Unknown 
Source)
        at 
java.base/jdk.internal.reflect.DelegatingMethodAccessorImpl.invoke(Unknown 
Source)
        at java.base/java.lang.reflect.Method.invoke(Unknown Source)
        at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.lambda$handleRpcInvocation$1(PekkoRpcActor.java:318)
        at 
org.apache.flink.runtime.concurrent.ClassLoadingUtils.runWithContextClassLoader(ClassLoadingUtils.java:83)
        at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcInvocation(PekkoRpcActor.java:316)
        at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleRpcMessage(PekkoRpcActor.java:229)
        at 
org.apache.flink.runtime.rpc.pekko.FencedPekkoRpcActor.handleRpcMessage(FencedPekkoRpcActor.java:88)
        at 
org.apache.flink.runtime.rpc.pekko.PekkoRpcActor.handleMessage(PekkoRpcActor.java:174)
        at 
org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:33)
        at 
org.apache.pekko.japi.pf.UnitCaseStatement.apply(CaseStatements.scala:29)
        at scala.PartialFunction.applyOrElse(PartialFunction.scala:127)
        at scala.PartialFunction.applyOrElse$(PartialFunction.scala:126)
        at 
org.apache.pekko.japi.pf.UnitCaseStatement.applyOrElse(CaseStatements.scala:29)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:175)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
        at scala.PartialFunction$OrElse.applyOrElse(PartialFunction.scala:176)
        at org.apache.pekko.actor.Actor.aroundReceive(Actor.scala:547)
        at org.apache.pekko.actor.Actor.aroundReceive$(Actor.scala:545)
        at 
org.apache.pekko.actor.AbstractActor.aroundReceive(AbstractActor.scala:229)
        at org.apache.pekko.actor.ActorCell.receiveMessage(ActorCell.scala:590)
        at org.apache.pekko.actor.ActorCell.invoke(ActorCell.scala:557)
        at org.apache.pekko.dispatch.Mailbox.processMailbox(Mailbox.scala:272)
        at org.apache.pekko.dispatch.Mailbox.run(Mailbox.scala:233)
        at org.apache.pekko.dispatch.Mailbox.exec(Mailbox.scala:245)
        at java.base/java.util.concurrent.ForkJoinTask.doExec(Unknown Source)
        at 
java.base/java.util.concurrent.ForkJoinPool$WorkQueue.topLevelExec(Unknown 
Source)
        at java.base/java.util.concurrent.ForkJoinPool.scan(Unknown Source)
        at java.base/java.util.concurrent.ForkJoinPool.runWorker(Unknown Source)
        at java.base/java.util.concurrent.ForkJoinWorkerThread.run(Unknown 
Source)
   Caused by: java.lang.RuntimeException: File deletion conflicts detected! 
Give up committing.
   
   Don't panic!
   Conflicts during commits are normal and this failure is intended to resolve 
the conflicts.
   Conflicts are mainly caused by the following scenarios:
   1. Multiple jobs are writing into the same partition at the same time, or 
you use STATEMENT SET to execute multiple INSERT statements into the same 
Paimon table.
      You'll probably see different base commit user and current commit user 
below.
      You can use 
https://paimon.apache.org/docs/master/maintenance/dedicated-compaction#dedicated-compaction-job
 to support multiple writing.
   2. You're recovering from an old savepoint, or you're creating multiple jobs 
from a savepoint.
      The job will fail continuously in this scenario to protect metadata from 
corruption.
      You can either recover from the latest savepoint, or you can revert the 
table to the snapshot corresponding to the old savepoint.
   
   Base commit user is: 8c4e22ba-1553-4021-bbee-944764ce41aa; Current commit 
user is: 0ac35332-87ee-4f0e-8143-f5dac9e6d523
   
   
   Base entries are:
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=2, 
totalBuckets=12, level=0, 
fileName=data-100a0c06-cda9-4e1a-bada-23357869888a-0.parquet, 
extraFiles=[data-100a0c06-cda9-4e1a-bada-23357869888a-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=8, 
totalBuckets=12, level=0, 
fileName=data-a8f1b651-8af1-40bb-a6e5-bd751e052fa3-0.parquet, 
extraFiles=[data-a8f1b651-8af1-40bb-a6e5-bd751e052fa3-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=4, 
totalBuckets=12, level=0, 
fileName=data-ecb6d0c3-9423-4259-a919-cd150b9e8fea-0.parquet, 
extraFiles=[data-ecb6d0c3-9423-4259-a919-cd150b9e8fea-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=10, 
totalBuckets=12, level=0, 
fileName=data-8bfad8ef-27aa-4293-8f38-5db74cd19ff4-0.parquet, 
extraFiles=[data-8bfad8ef-27aa-4293-8f38-5db74cd19ff4-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=3, 
totalBuckets=12, level=0, 
fileName=data-405288e8-e636-46b7-a698-e08efd900df3-0.parquet, 
extraFiles=[data-405288e8-e636-46b7-a698-e08efd900df3-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@acfb97e5, bucket=9, 
totalBuckets=12, level=0, 
fileName=data-1315369e-a10d-4a5f-a087-b1c8c1f32ca7-0.parquet, 
extraFiles=[data-1315369e-a10d-4a5f-a087-b1c8c1f32ca7-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   
   
   Changes are:
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@38567196, 
bucket=10, totalBuckets=12, level=0, 
fileName=data-f973d582-3a35-43e5-aed2-9c3af645bc6f-0.parquet, 
extraFiles=[data-f973d582-3a35-43e5-aed2-9c3af645bc6f-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@38567196, 
bucket=10, totalBuckets=12, level=0, 
fileName=data-f973d582-3a35-43e5-aed2-9c3af645bc6f-1.parquet, 
extraFiles=[data-f973d582-3a35-43e5-aed2-9c3af645bc6f-1.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@38567196, 
bucket=10, totalBuckets=12, level=0, 
fileName=data-f973d582-3a35-43e5-aed2-9c3af645bc6f-2.parquet, 
extraFiles=[data-f973d582-3a35-43e5-aed2-9c3af645bc6f-2.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@38567196, 
bucket=10, totalBuckets=12, level=0, 
fileName=data-f973d582-3a35-43e5-aed2-9c3af645bc6f-3.parquet, 
extraFiles=[data-f973d582-3a35-43e5-aed2-9c3af645bc6f-3.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@38567196, 
bucket=10, totalBuckets=12, level=0, 
fileName=data-f973d582-3a35-43e5-aed2-9c3af645bc6f-4.parquet, 
extraFiles=[data-f973d582-3a35-43e5-aed2-9c3af645bc6f-4.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@38567196, bucket=10, 
totalBuckets=12, level=0, 
fileName=data-7dd555aa-9eb0-4fcc-a68a-b9a22a769d8c-0.parquet, 
extraFiles=[data-7dd555aa-9eb0-4fcc-a68a-b9a22a769d8c-0.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=ADD, partition=org.apache.paimon.data.BinaryRow@38567196, bucket=10, 
totalBuckets=12, level=0, 
fileName=data-7dd555aa-9eb0-4fcc-a68a-b9a22a769d8c-1.parquet, 
extraFiles=[data-7dd555aa-9eb0-4fcc-a68a-b9a22a769d8c-1.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   {kind=DELETE, partition=org.apache.paimon.data.BinaryRow@faad274e, bucket=6, 
totalBuckets=12, level=0, 
fileName=data-ddb1f447-48a8-4b37-9174-1c7a6b725059-7.parquet, 
extraFiles=[data-ddb1f447-48a8-4b37-9174-1c7a6b725059-7.parquet.index], 
minKey=org.apache.paimon.data.BinaryRow@9c67b85d, 
maxKey=org.apache.paimon.data.BinaryRow@9c67b85d, externalPath=null}
   
   
   The entry list above are not fully displayed, please refer to 
taskmanager.log for more information.
        at 
org.apache.paimon.operation.FileStoreCommitImpl.createConflictException(FileStoreCommitImpl.java:1486)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.lambda$noConflictsOrFail$15(FileStoreCommitImpl.java:1314)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.noConflictsOrFail(FileStoreCommitImpl.java:1329)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.tryCommitOnce(FileStoreCommitImpl.java:964)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.tryCommit(FileStoreCommitImpl.java:782)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.commit(FileStoreCommitImpl.java:361)
        at 
org.apache.paimon.table.sink.TableCommitImpl.commitMultiple(TableCommitImpl.java:217)
        at 
org.apache.paimon.table.sink.TableCommitImpl.filterAndCommitMultiple(TableCommitImpl.java:256)
        at 
org.apache.paimon.flink.sink.StoreCommitter.filterAndCommit(StoreCommitter.java:121)
        at 
org.apache.paimon.flink.sink.CommitterOperator.commitUpToCheckpoint(CommitterOperator.java:213)
        at 
org.apache.paimon.flink.sink.CommitterOperator.endInput(CommitterOperator.java:186)
        at 
org.apache.flink.streaming.runtime.tasks.StreamOperatorWrapper.endOperatorInput(StreamOperatorWrapper.java:96)
        at 
org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.endInput(RegularOperatorChain.java:97)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:68)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:638)
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:973)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:917)
        at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:970)
        at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:949)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:763)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:575)
        at java.base/java.lang.Thread.run(Unknown Source)
   Caused by: java.lang.IllegalStateException: Trying to add file 
{partition=org.apache.paimon.data.BinaryRow@f459c4f, bucket=9, level=0, 
fileName=data-578040f5-8ea4-44a6-b100-ef64e07e0987-0.parquet, 
extraFiles=[data-578040f5-8ea4-44a6-b100-ef64e07e0987-0.parquet.index], 
embeddedIndex=null, externalPath=null} which is already added.
        at 
org.apache.paimon.utils.Preconditions.checkState(Preconditions.java:204)
        at org.apache.paimon.manifest.FileEntry.mergeEntries(FileEntry.java:193)
        at org.apache.paimon.manifest.FileEntry.mergeEntries(FileEntry.java:175)
        at 
org.apache.paimon.operation.FileStoreCommitImpl.noConflictsOrFail(FileStoreCommitImpl.java:1327)
        ... 20 more
   
   ### What doesn't meet your expectations?
   
   Dedicated compaction job keeps failing at the last subtask.
   
   Function call (same results for both CALLs):
   ①tableEnv.executeSql("CALL sys.compact(" +
                               "  `table` => '" + tableName + "', " +
                               "  partitions => 'dt="+dt+"', " +
                               "  compact_strategy => 'minor', " +
                               "  options => 
'target-file-size=128mb,sink.parallelism="+env.getParallelism()+"' " +
                               ");");
   
   ②tableEnv.executeSql("CALL sys.compact(" +
                               "  `table` => '" + tableName + "', " +
                               "  compact_strategy => 'minor', " +
                               " partition_idle_time => '"+days+"d', " +
                               "  options => 
'target-file-size=128mb,sink.parallelism="+env.getParallelism()+"' " +
                               ");");
   0. BucketedAppendTable with OSS storage,
   1. No other job is writing the table.
   2. Some partition worked,but no new snapshot created (maybe no file meet the 
compaction condition?) .
   3. Batch job started without savepoint.
   
   ### Anything else?
   
   _No response_
   
   ### Are you willing to submit a PR?
   
   - [ ] I'm willing to submit a PR!


-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to