[
https://issues.apache.org/jira/browse/FLINK-24064?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17407284#comment-17407284
]
Thomas Weise commented on FLINK-24064:
--------------------------------------
Exception restoring coordinator:
{code:java}
ERROR
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator
[] - Failed to reset the coordinator to checkpoint and start.2021-08-30
20:00:24,027 ERROR
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator
[] - Failed to reset the coordinator to checkpoint and
start.java.lang.NullPointerException: Source for index=1 not available at
org.apache.flink.util.Preconditions.checkNotNull(Preconditions.java:104)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.connector.base.source.hybrid.HybridSourceEnumeratorStateSerializer.lambda$serializerOf$0(HybridSourceEnumeratorStateSerializer.java:99)
~[?:?] at java.util.HashMap.computeIfAbsent(HashMap.java:1133) ~[?:?] at
org.apache.flink.connector.base.source.hybrid.HybridSourceEnumeratorStateSerializer.serializerOf(HybridSourceEnumeratorStateSerializer.java:95)
~[?:?] at
org.apache.flink.connector.base.source.hybrid.HybridSourceEnumeratorStateSerializer.deserializeV0(HybridSourceEnumeratorStateSerializer.java:89)
~[?:?] at
org.apache.flink.connector.base.source.hybrid.HybridSourceEnumeratorStateSerializer.deserialize(HybridSourceEnumeratorStateSerializer.java:72)
~[?:?] at
org.apache.flink.connector.base.source.hybrid.HybridSourceEnumeratorStateSerializer.deserialize(HybridSourceEnumeratorStateSerializer.java:34)
~[?:?] at
org.apache.flink.runtime.source.coordinator.SourceCoordinator.deserializeCheckpoint(SourceCoordinator.java:398)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.source.coordinator.SourceCoordinator.resetToCheckpoint(SourceCoordinator.java:301)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator$DeferrableCoordinator.resetAndStart(RecreateOnResetOperatorCoordinator.java:377)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator.lambda$resetToCheckpoint$6(RecreateOnResetOperatorCoordinator.java:136)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
java.util.concurrent.CompletableFuture.uniRunNow(CompletableFuture.java:815)
~[?:?] at
java.util.concurrent.CompletableFuture.uniRunStage(CompletableFuture.java:799)
~[?:?] at
java.util.concurrent.CompletableFuture.thenRun(CompletableFuture.java:2121)
~[?:?] at
org.apache.flink.runtime.operators.coordination.RecreateOnResetOperatorCoordinator.resetToCheckpoint(RecreateOnResetOperatorCoordinator.java:131)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.operators.coordination.OperatorCoordinatorHolder.resetToCheckpoint(OperatorCoordinatorHolder.java:273)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.checkpoint.CheckpointCoordinator.restoreStateToCoordinators(CheckpointCoordinator.java:1815)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.checkpoint.CheckpointCoordinator.restoreLatestCheckpointedStateInternal(CheckpointCoordinator.java:1577)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.checkpoint.CheckpointCoordinator.restoreSavepoint(CheckpointCoordinator.java:1642)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.tryRestoreExecutionGraphFromSavepoint(DefaultExecutionGraphFactory.java:163)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.DefaultExecutionGraphFactory.createAndRestoreExecutionGraph(DefaultExecutionGraphFactory.java:138)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.SchedulerBase.createAndRestoreExecutionGraph(SchedulerBase.java:342)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.SchedulerBase.<init>(SchedulerBase.java:190)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.DefaultScheduler.<init>(DefaultScheduler.java:122)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.scheduler.DefaultSchedulerFactory.createInstance(DefaultSchedulerFactory.java:132)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.jobmaster.DefaultSlotPoolServiceSchedulerFactory.createScheduler(DefaultSlotPoolServiceSchedulerFactory.java:110)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.jobmaster.JobMaster.createScheduler(JobMaster.java:340)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:317)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.internalCreateJobMasterService(DefaultJobMasterServiceFactory.java:107)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.runtime.jobmaster.factories.DefaultJobMasterServiceFactory.lambda$createJobMasterService$0(DefaultJobMasterServiceFactory.java:95)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
org.apache.flink.util.function.FunctionUtils.lambda$uncheckedSupplier$4(FunctionUtils.java:112)
~[flink-dist_2.12-1.13.2-stream1.jar:1.13.2-stream1] at
java.util.concurrent.CompletableFuture$AsyncSupply.run(CompletableFuture.java:1700)
[?:?] at
java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515) [?:?]
at java.util.concurrent.FutureTask.run(FutureTask.java:264) [?:?] at
java.util.concurrent.ScheduledThreadPoolExecutor$ScheduledFutureTask.run(ScheduledThreadPoolExecutor.java:304)
[?:?] at
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
[?:?] at
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
[?:?] at java.lang.Thread.run(Thread.java:829) [?:?] {code}
> HybridSource recovery from savepoint fails
> ------------------------------------------
>
> Key: FLINK-24064
> URL: https://issues.apache.org/jira/browse/FLINK-24064
> Project: Flink
> Issue Type: Bug
> Components: Connectors / Common
> Reporter: Thomas Weise
> Assignee: Thomas Weise
> Priority: Major
>
> Recovery fails because underlying source and split deserializers are not
> initialized in the restore code path. This requires deferred deserialization
> after the current source has been set.
--
This message was sent by Atlassian Jira
(v8.3.4#803005)