+1 from my size.

I think it will be a good feature.

Best
-- 
Louis
Email:xu_soft39211...@163.com

> On 30 May 2019, at 15:57, Tzu-Li (Gordon) Tai <tzuli...@apache.org> wrote:
> 
> The name "Savepoint Connector" might indeed be not that good, as it doesn't
> point out the fact that with the current design, all kinds of snapshots
> (savepoint / full or incremental checkpoints) can be read.
> 
> @Paul
> That would be a very valid requirement. Querying the list of existing
> operator ids should be straight forward, as that information is in the
> snapshot metadata file.
> However, querying state names / state structure / state type would
> currently be impossible without also reading the state itself, as that
> information isn't globally available and can only be known when each key
> group is being read. We could potentially make those information available
> in the snapshot metadata file, but that would require more work. I think
> that can be a next step once we have an initial version.
> 
> Cheers,
> Gordon
> 
> On Thu, May 30, 2019 at 1:21 PM Paul Lam <paullin3...@gmail.com> wrote:
> 
>> Hi Seth,
>> 
>> Sorry for the confusion. I mean currently we need to know the operator id,
>> state name and the state type (eg. ListState, MapState) beforehand to get
>> the states. Is possible that we can perform a scan to get all existing
>> operator ids or state names in the savepoint? It would be good to know what
>> states are in the savepoint before we get to a specific state.
>> 
>> For example, if we analyze a savepoint created weeks ago, and the
>> corresponding job has been modified since that, say, moved from KafkaSink
>> to KinesisSink, so we are not sure whether we have the Kafka sink states or
>> the Kinesis sink states in the savepoint and might need to try twice to get
>> the right one.
>> 
>> I’m not familiar with the savepoint formats, so pardon me if it’s a dumb
>> question.
>> 
>> Best,
>> Paul Lam
>> 
>> 在 2019年5月30日,11:09,Seth Wiesman <sjwies...@gmail.com> 写道:
>> 
>> Hi Paul,
>> 
>> I’m not following, could you provide and example of the kind of operation
>> your describing?
>> 
>> Seth
>> 
>> On May 29, 2019, at 7:37 PM, Paul Lam <paullin3...@gmail.com> wrote:
>> 
>> Hi Seth,
>> 
>> +1 from my side.
>> 
>> I was wondering if we can add a reader method to provide a full view of
>> the states instead of the state of a specific operator? It would be helpful
>> when there is some unrestored states of a previously removed operator in
>> the savepoint.
>> 
>> Best,
>> Paul Lam
>> 
>> 在 2019年5月30日,09:55,vino yang <yanghua1...@gmail.com> 写道:
>> 
>> Hi Seth,
>> 
>> Glad to see this FLIP, big +1 for this feature!
>> 
>> Best,
>> Vino
>> 
>> Seth Wiesman <sjwies...@gmail.com> 于2019年5月30日周四 上午7:14写道:
>> 
>> Hey Everyone!
>> ​
>> Gordon and I have been discussing adding a savepoint connector to flink
>> for reading, writing and modifying savepoints.
>> ​
>> This is useful for:
>> ​
>> Analyzing state for interesting patterns
>> Troubleshooting or auditing jobs by checking for discrepancies in state
>> Bootstrapping state for new applications
>> Modifying savepoints such as:
>>     Changing max parallelism
>>     Making breaking schema changes
>>     Correcting invalid state
>> ​
>> We are looking forward to your feedback!
>> ​
>> This is the FLIP:
>> ​
>> 
>> 
>> https://cwiki.apache.org/confluence/display/FLINK/FLIP-43%3A+Savepoint+Connector
>> 
>> Seth
>> 
>> 
>> 
>> 
>> 
>> 


Reply via email to