[ 
https://issues.apache.org/jira/browse/FLINK-38402?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=18031264#comment-18031264
 ] 

Roman Khachatryan commented on FLINK-38402:
-------------------------------------------

> There was a suspicious 'Attempt to reference unknown state: 
> 525e0fb0-bd67-3b89-84ee-feede17a755a', 

> which I guess may cause the checkpoint cannot finish until expired. Any 
> thoughts? [~roman]

 
[~zakelly] good point, that would cause a checkpoint timeout.
At first, I suspected changelog state backend causing "unknown state 
reference"; but 
[one|https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=70172&view=logs&j=39d5b1d5-3b41-54dc-6458-1e2ddd1cdcf3&t=115e5c38-6efb-5006-4921-5e2851da71ef&l=6898]
 of the failures doesn't seem to have it enabled.
{code:java}

06:38:11,749 [                main] INFO  
org.apache.flink.configuration.Configuration                 [] - Config uses 
fallback configuration key 'jobmanager.rpc.address' instead of key 
'rest.address'
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for execution.checkpointing.unaligned.enabled
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected PT0S for execution.checkpointing.aligned-checkpoint-timeout
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for execution.checkpointing.cleaner.parallel-mode
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for execution.checkpointing.unaligned.interruptible-timers.enabled
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for execution.checkpointing.snapshot-compression
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for execution.checkpointing.file-merging.enabled
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected true for state.backend.rocksdb.use-ingest-db-restore-mode
06:38:11,750 [                main] INFO  
org.apache.flink.runtime.testutils.PseudoRandomValueSelector [] - Randomly 
selected 1 for table.exec.unbounded-over.version...06:38:35,797 [    Checkpoint 
Timer] INFO  org.apache.flink.runtime.checkpoint.CheckpointRequestDecider [] - 
checkpoint request time in queue: 7500
06:38:35,798 [    Checkpoint Timer] INFO  
org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Triggering 
checkpoint 113 (type=CheckpointType{name='Checkpoint', 
sharingFilesStrategy=FORWARD_BACKWARD}) @ 1760164715797 for job 
4e32249219f87f49c61d8295cc1d82f6.
06:38:35,840 [jobmanager-io-thread-1] WARN  
org.apache.flink.runtime.jobmaster.JobMaster                 [] - Error while 
processing AcknowledgeCheckpoint message
java.lang.IllegalStateException: Attempt to reference unknown state: 
bad5aa62-dea0-359a-b751-22c6e28a651d
      at org.apache.flink.util.Preconditions.checkState(Preconditions.java:193) 
~[flink-core-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.state.SharedStateRegistryImpl.registerReference(SharedStateRegistryImpl.java:97)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.state.SharedStateRegistry.registerReference(SharedStateRegistry.java:54)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.state.IncrementalRemoteKeyedStateHandle.registerSharedStates(IncrementalRemoteKeyedStateHandle.java:289)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.checkpoint.OperatorSubtaskState.registerSharedState(OperatorSubtaskState.java:248)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.checkpoint.OperatorSubtaskState.registerSharedStates(OperatorSubtaskState.java:224)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.checkpoint.TaskStateSnapshot.registerSharedStates(TaskStateSnapshot.java:193)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.checkpoint.CheckpointCoordinator.receiveAcknowledgeMessage(CheckpointCoordinator.java:1246)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$acknowledgeCheckpoint$2(ExecutionGraphHandler.java:109)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
org.apache.flink.runtime.scheduler.ExecutionGraphHandler.lambda$processCheckpointCoordinatorMessage$4(ExecutionGraphHandler.java:139)
 ~[flink-runtime-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at org.apache.flink.util.MdcUtils.lambda$wrapRunnable$1(MdcUtils.java:70) 
~[flink-core-2.2-SNAPSHOT.jar:2.2-SNAPSHOT]
      at 
java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
 [?:?]
      at 
java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
 [?:?]
      at java.base/java.lang.Thread.run(Thread.java:833) [?:?]...Test 
testPreAggregatedSlidingTimeWindow[statebackend type 
=ROCKSDB_INCREMENTAL](org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase)
 failed with:
  java.lang.AssertionError: Job execution failed.
          at org.junit.Assert.fail(Assert.java:89)
          at 
org.apache.flink.test.checkpointing.EventTimeWindowCheckpointingITCase.testPreAggregatedSlidingTimeWindow(EventTimeWindowCheckpointingITCase.java:738)
 {code}
>From the enabled options, `execution.checkpointing.file-merging.enabled` seems 
>to be relevant AND the related code was changed before this ticket was created 
>(in [https://github.com/apache/flink/pull/26977]).
Do you think it can be the root cause?

> EventTimeWindowCheckpointingITCase failed in test_cron_azure tests
> ------------------------------------------------------------------
>
>                 Key: FLINK-38402
>                 URL: https://issues.apache.org/jira/browse/FLINK-38402
>             Project: Flink
>          Issue Type: Bug
>          Components: Tests
>    Affects Versions: 2.2.0
>            Reporter: Ruan Hang
>            Priority: Major
>
> Details:
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=69810&view=logs&j=39d5b1d5-3b41-54dc-6458-1e2ddd1cdcf3&t=115e5c38-6efb-5006-4921-5e2851da71ef



--
This message was sent by Atlassian Jira
(v8.20.10#820010)

Reply via email to