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

Tzu-Li (Gordon) Tai edited comment on FLINK-12047 at 5/30/19 5:09 AM:
----------------------------------------------------------------------

[~gyfora] I think with the current design of FLIP-43, the savepoint connector 
should be able to read any kind of snapshot, since it is essentially delegating 
the restore of state to an actual {{StreamTask}} and therefore recognizes 
different kinds of state handles transparently.

{code}
Even if we support only writing states in a savepoint format as Flink restores 
operators independently you can keep the checkpoint state of other operators 
and have a savepoint format for only the modified one.
{code}

Not entirely following this part. Do you mean that only the touched operator 
state will be rewritten with a savepoint format, while other untouched 
operators will remain with a checkpoint format?


was (Author: tzulitai):
[~gyfora] I think with the current design of FLIP-43, the savepoint connector 
should be able to read any kind of snapshot, since it is essentially delegating 
the restore of state to an actual `StreamTask` and therefore recognizes 
different state handles transparently.

{code}
Even if we support only writing states in a savepoint format as Flink restores 
operators independently you can keep the checkpoint state of other operators 
and have a savepoint format for only the modified one.
{code}

Not entirely following this part. Do you mean that only the touched operator 
state will be rewritten with a savepoint format, while other untouched 
operators will remain with a checkpoint format?

> Savepoint connector to read / write / process savepoints
> --------------------------------------------------------
>
>                 Key: FLINK-12047
>                 URL: https://issues.apache.org/jira/browse/FLINK-12047
>             Project: Flink
>          Issue Type: New Feature
>          Components: API / DataStream, Runtime / State Backends
>            Reporter: Tzu-Li (Gordon) Tai
>            Assignee: Seth Wiesman
>            Priority: Major
>
> This JIRA tracks the ongoing efforts and discussions about a means to read / 
> write / process state in savepoints.
> There are already two known existing works (that was mentioned already in the 
> mailing lists) related to this:
> 1. Bravo [1]
> 2. https://github.com/sjwiesman/flink/tree/savepoint-connector
> Essentially, the two tools both provide a connector to read or write a Flink 
> savepoint, and allows to utilize Flink's processing APIs for querying / 
> processing the state in the savepoint.
> We should try to converge the efforts on this, and have a savepoint connector 
> like this in Flink.
> With this connector, the high-level benefits users should be able to achieve 
> with it are:
> 1. Create savepoints using existing data from other systems (i.e. 
> bootstrapping a Flink job's state with data in an external database).
> 2. Derive new state using existing state
> 3. Query state in savepoints, for example for debugging purposes
> 4. Migrate schema of state in savepoints offline, compared to the current 
> more limited approach of online migration on state access.
> 5. Change max parallelism of jobs, or any other kind of fixed configuration, 
> such as operator uids.
> [1] https://github.com/king/bravo



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to