>
> So, can't we extract all previous savepoint data  by using
> ExistingSavepoint?


You can extract all of the data from any specific savepoint. Or nearly all
data, anyway. There is at least one corner case that isn't covered --
ListCheckpointed state -- which has been deprecated and isn't supported by
the savepoint API.

David

On Fri, Apr 30, 2021 at 5:42 PM Abdullah bin Omar <
abdullahbinoma...@gmail.com> wrote:

> Hi,
>
> So, can't we extract all previous savepoint data  by using
> ExistingSavepoint?
>
>
> Thank you
>
>
>
>
>
>
> On Fri, Apr 30, 2021 at 10:25 AM David Anderson <dander...@apache.org>
> wrote:
>
>> Abdullah,
>>
>> The example you are studying -- the one using the state processor API --
>> can be used with any retained checkpoint or savepoint created while running
>> the RidesAndFaresSolution job. But this is a very special use of
>> checkpoints and savepoints that shows how to extract data from them.
>>
>> Normally the state processor API is used with savepoints, and not with
>> checkpoints. This example uses checkpoints so that the example can be
>> easily run from the IDE, without requiring a local flink installation.
>>
>> The normal use for checkpoints is for failure recovery, while savepoints
>> are typically used for redeployments and rescaling -- and in these cases
>> the state processor API is not involved. You would use "flink run -s ..."
>> on the command line to manually resume from a checkpoint or savepoint, and
>> in the case of a job failure, the restart will happen automatically.
>>
>> The flink operations playground [1] is a great way to gain more
>> understanding of these aspects of flink.
>>
>> [1]
>> https://ci.apache.org/projects/flink/flink-docs-stable/try-flink/flink-operations-playground.html
>>
>> Best regards,
>> David
>>
>> On Fri, Apr 30, 2021 at 1:56 PM Abdullah bin Omar <
>> abdullahbinoma...@gmail.com> wrote:
>>
>>> Hi,
>>>
>>> Please answer me some of my below question whether my understanding
>>> correct or not, and please answer the direct ask questions.
>>>
>>> *Question no 1 (about dependency):*
>>>
>>> *What is dependency (in pom.xml) for the org.apache.flink.training?*
>>>
>>> I am trying to *import* 
>>> org.apache.flink.training.exercises.common.sources.TaxiFareGenerator;
>>> However, it can not resolve.
>>>
>>> [note that, I am using the group id: <groupId>org.apache.flink</groupId>
>>>
>>>
>>> *Question No 2 (which one is being load to an existing savepoint):*
>>>
>>>
>>> According to my understanding after reading [1], the name
>>> "ExistingSavepoint" looks like that it will restore all previous savepoint.
>>> However, according to [2], the input file is only a checkpointed file.
>>>
>>>
>>> *(i)* *is that mean that we can only load the last checkpointed file
>>> (in case of job failure) by using the ExistingSavepoint to restart the job
>>> where it fails?*
>>>
>>>
>>> *(ii)* *and there is no option to load all previous savepoint. is this
>>> correct?*
>>>
>>>
>>>
>>> *Question No 3 (about loading an existing savepoint):*
>>>
>>> ExecutionEnvironment bEnv = ExecutionEnvironment.
>>> *getExecutionEnvironment*();
>>>
>>> ExistingSavepoint sp = Savepoint.*load*(bEnv, "hdfs://path/", new
>>> MemoryStateBackend);
>>>
>>>
>>>
>>> This is the code for loading an existing savepoint. However, I configure
>>> a file location in flink conf to save the savepoint. So then, each time the
>>> job is running. I use a command in the terminal, ./bin/flink savepoint jobid
>>>
>>> and the savepointed file saved in the file location (that is set up in
>>> flink conf).
>>>
>>>
>>> In this case, to load the savepoint, file location will be the location
>>> that set up in the flink conf and FileSystemBackend will have to use
>>> instead of MemoryStateBackend. *is this correct?*
>>>
>>>
>>>
>>>
>>> [1]
>>> https://ci.apache.org/projects/flink/flink-docs-stable/dev/libs/state_processor_api.html
>>>
>>> [2]
>>> https://github.com/ververica/flink-training/blob/master/state-processor/src/main/java/com/ververica/flink/training/exercises/ReadRidesAndFaresSnapshot.java
>>>
>>>
>>>
>>>
>>> Thank you
>>>
>>>
>>>
>>>
>>>
>>>
>>> On Fri, Apr 23, 2021 at 10:10 AM David Anderson <dander...@apache.org>
>>> wrote:
>>>
>>>> Abdullah,
>>>>
>>>> ReadRidesAndFaresSnapshot [1] is an example that shows how to use the
>>>> State Processor API to display the contents of a snapshot taken while
>>>> running RidesAndFaresSolution [2].
>>>>
>>>> Hopefully that will help you get started.
>>>>
>>>> [1]
>>>> https://github.com/ververica/flink-training/blob/master/state-processor/src/main/java/com/ververica/flink/training/exercises/ReadRidesAndFaresSnapshot.java
>>>> [2]
>>>> https://github.com/ververica/flink-training/blob/master/rides-and-fares/src/solution/java/org/apache/flink/training/solutions/ridesandfares/RidesAndFaresSolution.java
>>>>
>>>> Best regards,
>>>> David
>>>>
>>>> On Fri, Apr 23, 2021 at 3:32 PM Abdullah bin Omar <
>>>> abdullahbinoma...@gmail.com> wrote:
>>>>
>>>>> Hi,
>>>>>
>>>>> Thank you for your reply.
>>>>>
>>>>> I want to read the previous snapshot (if needed) at the time of
>>>>> operation. In [1], there is a portion:
>>>>>
>>>>> DataSet<Integer> listState  = savepoint.readListState<>(
>>>>>     "my-uid",
>>>>>     "list-state",
>>>>>     Types.INT);
>>>>>
>>>>>
>>>>> here, will the function savepoint.readliststate<> () work to read the
>>>>> previous snapshot?  If it is, then is the filename of a savepoint file
>>>>> similar to my-uid?
>>>>>
>>>>> [1]
>>>>> https://ci.apache.org/projects/flink/flink-docs-stable/dev/libs/state_processor_api.html
>>>>>
>>>>> Thank you
>>>>>
>>>>>
>>>>>
>>>>>
>>>>> On Fri, Apr 23, 2021 at 1:11 AM Matthias Pohl <matth...@ververica.com>
>>>>> wrote:
>>>>>
>>>>>> What is it you're trying to achieve in general? The JavaDoc of
>>>>>> MetadataV2V3SerializerBase provides a description on the format of the
>>>>>> file. Theoretically, you could come up with custom code using the Flink
>>>>>> sources to parse the content of the file. But maybe, there's another way 
>>>>>> to
>>>>>> accomplish what you're trying to do.
>>>>>>
>>>>>> Matthias
>>>>>>
>>>>>> [1]
>>>>>> https://github.com/apache/flink/blob/adaaed426c2e637b8e5ffa3f0d051326038d30aa/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java#L83
>>>>>>
>>>>>> On Thu, Apr 22, 2021 at 7:53 PM Abdullah bin Omar <
>>>>>> abdullahbinoma...@gmail.com> wrote:
>>>>>>
>>>>>>> Hi,
>>>>>>>
>>>>>>> I have a savepoint or checkpointed file from my task. However, the
>>>>>>> file is binary. I want to see what the file contains.
>>>>>>>
>>>>>>> How is it possible to see what information the file has (or how it
>>>>>>> is possible to make it human readable?)
>>>>>>>
>>>>>>> Thank you
>>>>>>>
>>>>>>> On Thu, Apr 22, 2021 at 10:19 AM Matthias Pohl <
>>>>>>> matth...@ververica.com> wrote:
>>>>>>>
>>>>>>>> Hi Abdullah,
>>>>>>>> the metadata file contains handles to the operator states of the
>>>>>>>> checkpoint [1]. You might want to have a look into the State Processor 
>>>>>>>> API
>>>>>>>> [2].
>>>>>>>>
>>>>>>>> Best,
>>>>>>>> Matthias
>>>>>>>>
>>>>>>>> [1]
>>>>>>>> https://github.com/apache/flink/blob/adaaed426c2e637b8e5ffa3f0d051326038d30aa/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java#L83
>>>>>>>> [2]
>>>>>>>> https://ci.apache.org/projects/flink/flink-docs-stable/dev/libs/state_processor_api.html
>>>>>>>>
>>>>>>>> On Thu, Apr 22, 2021 at 4:57 PM Abdullah bin Omar <
>>>>>>>> abdullahbinoma...@gmail.com> wrote:
>>>>>>>>
>>>>>>>>> Hi,
>>>>>>>>>
>>>>>>>>> (1) what 's the snapshot metadata file (binary) contains ? is it
>>>>>>>>> possible to read the snapshot metadata file by using Flink 
>>>>>>>>> Deserialization?
>>>>>>>>>
>>>>>>>>> (2) is there any function that can be used to see the previous
>>>>>>>>> states on time of operation?
>>>>>>>>>
>>>>>>>>> Thank you
>>>>>>>>>
>>>>>>>>
>>>>>>

Reply via email to