Hi Kostas/Congxian:
Thanks fo your response.  
Based on your feedback, I found that I had missed adding uid to one of the 
stateful operators and correcting that resolved the issue.  I still have 
stateless operators which I have no uid specified in the application.
So, I thought that adding uid was optional and if we don't add it and run 
another instance of the same app from a savepoint or checkpoint, it will pick 
up the state based on the generated uid.  Is that a correct understanding ?  
Also, if some stateful operators have uids but some don't, will it pick up the 
state for the operators with uid and the non-uid (using the generated uid) ones 
provided the application has not changed ?
Thanks again for your response.
Mans
    On Monday, November 25, 2019, 09:24:42 AM EST, Congxian Qiu 
<qcx978132...@gmail.com> wrote:  
 
 Hi
The problem is that the specified uid did not in the new job.1. As far as I 
know, the answer is yes. There are some operators have their own state(such as 
window state), could you please share the minimal code of your job?2.*truely* 
stateless operator do not need to have uid, but for the reason described in the 
above, assign uid to all operators is recommended.3. if the previous job is 
still there, I'm not sure we can find the operatorId in the UI easily, maybe 
other people can answer the question.4. for this purpose, maybe you can debug 
the savepoint meta with the new job locally, maybe 
CheckpointMetadataLoadingTest can help.5. for this problem, 1.9 is same as 1.6

Best,Congxian

Kostas Kloudas <kklou...@gmail.com> 于2019年11月25日周一 下午9:42写道:

As a side note, I am assuming that you are using the same Flink Job
before and after the savepoint and the same Flink version.
Am I correct?

Cheers,
Kostas

On Mon, Nov 25, 2019 at 2:40 PM Kostas Kloudas <kklou...@gmail.com> wrote:
>
> Hi Singh,
>
> This behaviour is strange.
> One thing I can recommend to see if the two jobs are identical is to
> launch also the second job without a savepoint,
> just start from scratch, and simply look at the web interface to see
> if everything is there.
>
> Also could you please provide some code from your job, just to see if
> there is anything problematic with the application code?
> Normally there should be no problem with not providing UIDs for some
> stateless operators.
>
> Cheers,
> Kostas
>
> On Sat, Nov 23, 2019 at 11:16 AM M Singh <mans2si...@yahoo.com> wrote:
> >
> >
> > Hey Folks:
> >
> > Please let me know how to resolve this issue since using 
> > --allowNonRestoredState without knowing if any state will be lost seems 
> > risky.
> >
> > Thanks
> > On Friday, November 22, 2019, 02:55:09 PM EST, M Singh 
> > <mans2si...@yahoo.com> wrote:
> >
> >
> > Hi:
> >
> > I have a flink application in which some of the operators have uid and name 
> > and some stateless ones don't.
> >
> > I've taken a save point and tried to start another instance of the 
> > application from a savepoint - I get the following exception which 
> > indicates that the operator is not available to the new program even though 
> > the second job is the same as first but just running from the first jobs 
> > savepoint.
> >
> > Caused by: java.lang.IllegalStateException: Failed to rollback to 
> > checkpoint/savepoint 
> > s3://mybucket/state/savePoint/mysavepointfolder/66s4c6402d7532801287290436fa9fadd/savepoint-664c64-fa235d26d379.
> >  Cannot map checkpoint/savepoint state for operator 
> > d1a56c5a9ce5e3f1b03e01cac458bb4f 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.
> >
> > at 
> > org.apache.flink.runtime.checkpoint.Checkpoints.loadAndValidateCheckpoint(Checkpoints.java:205)
> >
> > at 
> > org.apache.flink.runtime.checkpoint.CheckpointCoordinator.restoreSavepoint(CheckpointCoordinator.java:1102)
> >
> > at 
> > org.apache.flink.runtime.jobmaster.JobMaster.tryRestoreExecutionGraphFromSavepoint(JobMaster.java:1219)
> >
> > at 
> > org.apache.flink.runtime.jobmaster.JobMaster.createAndRestoreExecutionGraph(JobMaster.java:1143)
> >
> > at org.apache.flink.runtime.jobmaster.JobMaster.<init>(JobMaster.java:294)
> >
> > at 
> > org.apache.flink.runtime.jobmaster.JobManagerRunner.<init>(JobManagerRunner.java:157)
> >
> > ... 10 more
> >
> >
> >
> > I've tried to start an application instance from the checkpoint too of the 
> > first instance but it gives the same exception indicating that the operator 
> > is not available.
> >
> > Questions:
> >
> > 1. If this a problem because some of the operators don't have uid ?
> > 2. Is it required to have uids even for stateless operators like simple map 
> > or filter operators ?
> > 3. Is there a way to find out which operator is not available in the new 
> > application even though I am running the same application ?
> > 4. Is there a way to figure out if this is the only missing operator or are 
> > there others whose mapping is missing for the second instance run ?
> > 5. Is this issue resolved in Apache Flink 1.9 (since I am still using Flink 
> > 1.6)
> >
> > If there any additional pointers please let me know.
> >
> > Thanks
> >
> > Mans
> >
> >

  

Reply via email to