I'm confident I'm hitting a bug, I guess I'm the first one trying this
recovery in the standalone mode :-D

Created https://issues.apache.org/jira/browse/FLINK-29633

On Thu, Oct 13, 2022 at 8:45 AM Yaroslav Tkachenko <yaros...@goldsky.com>
wrote:

> Thanks folks, I understand this can be a limitation when redeploying. I
> did try to delete my job and start it from scratch using
> "initialSavepointPath"... and I got the same issue. Going to investigate
> this more today.
>
> On Thu, Oct 13, 2022 at 12:18 AM Evgeniy Lyutikov <eblyuti...@avito.ru>
> wrote:
>
>> The problem is that changing the FlinkDeployment specification (new jar
>> version, changing pod resources, etc.) for JobManager is just a restart.
>>
>> 2022-09-16 09:30:52,526 INFO
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Restoring
>> job 00000000000000000000000000000000 from Checkpoint 34 @ 1663320593326 for
>> 00000000000000000000000000000000 located at
>> s3p://flink-checkpoints/k8s-checkpoint-test-k8s-deploy/00000000000000000000000000000000/chk-34.
>> 2022-09-16 09:30:52,624 INFO
>> org.apache.flink.runtime.dispatcher.StandaloneDispatcher     [] - Job
>> 00000000000000000000000000000000 reached terminal state FAILED.
>> org.apache.flink.runtime.client.JobInitializationException: Could not
>> start the JobMaster.
>> Caused by: java.util.concurrent.CompletionException:
>> java.lang.IllegalStateException: There is no operator for the state
>> f215196137eeb29b6f14c1ac14a1dc9f
>> Caused by: java.lang.IllegalStateException: There is no operator for the
>> state f215196137eeb29b6f14c1ac14a1dc9f
>>
>> After starting, it restores everything from the saved HA metadata saved
>> in the configmap (jobgraph, etc.).
>> The only correct method for us was to completely delete the
>> FlinkDeployment object and create a new one with initialSavepointPath and
>> allowNonRestoredState.
>> After that, the startup log looks a little different
>>
>> 2022-09-16 10:30:52,624 INFO
>> org.apache.flink.runtime.checkpoint.CheckpointCoordinator    [] - Restoring
>> job 00000000000000000000000000000000 from Savepoint 34 @ 0 for
>> 00000000000000000000000000000000 located at
>> s3p://flink-checkpoints/k8s-checkpoint-test-k8s-deploy/00000000000000000000000000000000/chk-34.
>>
>> ------------------------------
>> *От:* Gyula Fóra <gyula.f...@gmail.com>
>> *Отправлено:* 13 октября 2022 г. 13:19:54
>> *Кому:* Yaroslav Tkachenko
>> *Копия:* user
>> *Тема:* Re: allowNonRestoredState doesn't seem to be working
>>
>> Hi!
>>
>> If you have last-state upgrade mode configured it may happen that the
>> allowNonRestoredState config is ignored by Flink (as the last-state upgrade
>> mechanism somewhat bypasses the regular submission).
>>
>> Worst case scenario, you can suspend the deployment, manually record the
>> last checkpoint/savepoint path. Then delete the FlinkDeployment and
>> recreate it with the initialSavepointPath set to your checkpoint.
>>
>> Cheers,
>> Gyula
>>
>> On Thu, Oct 13, 2022 at 7:36 AM Yaroslav Tkachenko <yaros...@goldsky.com>
>> wrote:
>>
>>> Hey everyone,
>>>
>>> I'm trying to redeploy an application using a savepoint. The new version
>>> of the application has a few operators with new uids and a few operators
>>> with the old uids. I'd like to keep the state for the old ones.
>>>
>>> I passed the allowNonRestoredState flag (using Apache Kubernetes
>>> Operator actually) and I can confirm that
>>> "execution.savepoint.ignore-unclaimed-state" is "true" after that.
>>>
>>> However, the application still fails with the following exception:
>>>
>>> "java.lang.IllegalStateException: Failed to rollback to
>>> checkpoint/savepoint s3p://<REDACTED>. Cannot map checkpoint/savepoint
>>> state for operator d9ea0f9654a3395802138c72c1bfd35b to the new program,
>>> because the operator is not available in the new program. If you want to
>>> allow to skip this, you can set the --allowNonRestoredState option on the
>>> CLI."
>>>
>>> Is there a situation where allowNonRestoredState may not work? Thanks.
>>>
>>
>> * ------------------------------ *“This message contains confidential
>> information/commercial secret. If you are not the intended addressee of
>> this message you may not copy, save, print or forward it to any third party
>> and you are kindly requested to destroy this message and notify the sender
>> thereof by email.
>> Данное сообщение содержит конфиденциальную информацию/информацию,
>> являющуюся коммерческой тайной. Если Вы не являетесь надлежащим адресатом
>> данного сообщения, Вы не вправе копировать, сохранять, печатать или
>> пересылать его каким либо иным лицам. Просьба уничтожить данное сообщение и
>> уведомить об этом отправителя электронным письмом.”
>>
>

Reply via email to