[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-08-18 Thread Echo Lee (Jira)


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

Echo Lee commented on FLINK-25360:
--

[~liufangqi] Thank you very much for coming up with this idea. We recently want 
to do something similar to simplify the query state, especially for sql jobs. 
[~yunta] [~liufangqi] Would you like to know what is the current progress?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: chenfengLiu
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-01-06 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] OK, THX, I will put the design doc link in this issue comment later.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-01-06 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

[~liufangqi] I suggest to create a design doc via google doc first. And after 
several rounds of review, let's create a FLIP and launch a discussion then.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-01-05 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] THX for your reply, do you mean that I should write a proposal in FLIP 
first? As I saw, I can not create page there. Could you help to give me a 
guide? I will drive this work.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-01-05 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

[~liufangqi] I think we could at least to write a discuss email for persisting 
addition information in state descriptor. For the 1st problem, I think you 
could at least provide a proposal for how to persisting the state meta data in 
checkpoint meta. From my mind, this could require to add some additional 
interfaces.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2022-01-05 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] Hi, could you help to push the thread that we metioned before?

1. Could we write meta state info in savepoint meta?
2. Whether we should persist addition info such as default value, 
reducingFunction to persist the whole state descriptor?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Yue Ma (Jira)


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

Yue Ma commented on FLINK-25360:



> Could we write meta state info in savepoint meta?

[~yunta] [~liufangqi]   I'm really glad to see the Community doing this kind of 
discussion . Actually , we also did the same thing  (adding meta information 
such as state desc to the savepoint)  in the internal Flink version of 
ByteDance.The reason we did this in the first place was because now users need 
to re-register the state using state-processor-api to query the state. We think 
this is too diffcult for users. So we did some simple work to make it easier 
for users to query state (such as add state meta to Savepoint or use Flink 
Batch sql for state Query .so I personally agree with this idea.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] THX for your reply, actually I don't understand why RocksDB 
incremenatl checkpoints already have enough information in checkpoint meta. As 
I see, there are still not enough info in RocksDB incremenatl checkpoint meta 
such as UDF.

Reply to your two lines:
 # Is it means all state meta info will be stored in the checkpoint / savepoint 
meta, if the discussion is agreed.
 # How to start the discussion threads in dev mailing list? Could you help to 
give a guide?

Wish all of you a happy new year and a merry Christmas in advance.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

[~liufangqi] Since RocksDB incremenatl checkpoints already have enough 
information in checkpoint meta, could you try to use state processor API to 
achive your goal on that?

Your request actually has two problems, and I think they deserve two different 
discussion threads in dev mailling list:
 # Could we write meta state info in savepoint meta. [~dwysakowicz] what do you 
think of this especially you have already done the unified savepoint format 
work.
 # Whether we should persist addition info such as default value, 
reducingFunction to persist the whole state descriptor. I prefer to store them. 
[~tzulitai]  what do you think of this?

BTW, since many guys are on Christmas vacation, maybe we can launch discussions 
after new year's day.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~sjwiesman] THX for your reply, so should we add enough info to checkpoint 
meta for provided the power of self-described, whether this question worth to 
discuss? [~yunta] 

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Seth Wiesman (Jira)


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

Seth Wiesman commented on FLINK-25360:
--

[~yunta] there isn't any way to do this with the state processor api beyond 
what's already been discussed. The checkpoint metadata does not contain enough 
information to rewrite the snapshot automatically. 

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] As I see, exactly, raw state are used little. But there are still some 
important raw state which is used (PS:timer state).

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

Raw state is not recomended to use anymore, and flink would only persist the 
raw state output stream content.

 

[~sjwiesman] , do you have suggestion for this request to just modify the max 
paralleilsm via state processor API?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] Yeah, your understand is right, but changing the max paralleilsm seems 
like not simple, cause we need to rewrite the keyed state(we may even need to 
rewrite the operator state, on current state-processor-api version).

Actually we need all the info that used to build the 
org.apache.flink.api.common.state.StateDescriptor object. For example, state 
type (list / map or reduce), TypeSerializer, default value etc.

BTW, how we can mock the raw state generate?

 

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

It seems you're trying to read checkpoint to modify previous checkpoint without 
knowing the jobgraph (does not know what the operator is), right? If only wants 
to change the max paralleilsm, things could be much simpler.

 

Except from the agg function, what else do you need?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-22 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] Hi, THX, in my case, we want to read the checkpoint, change the 
maxParallelism, then store in the new checkpoint. And state-processor-api need 
we to build a dataSet application which means we need to be aware of all the 
state desc to write the UDF of the state-processor-api. And what we had is just 
the checkpoint metadata.

Reply to your second words:

Certainly, flink store the state's Metadata which contains state name / value 
serializer / key serializer/ namespace serializer, but it's not enough to build 
a state desc. (PS: the state desc which i mentioned is the 
org.apache.flink.api.common.state.StateDescriptor Object)

BTW, state-processor-api actually provide the power that we can handle the 
state, but we should mock the stream operator which make us so hard to generate 
the state and the checkpoint metadata.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-21 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

[~liufangqi], I think your concerns have two different problems:
# AggregatingStateDescriptor does not store the aggFunction on snapshot. This 
indeed occurs for a long time, and I think this deserve a disscussion in 
community mailling list. I prefer to store this in state descriptior.
# Checkpoint meta does not contains the state descriptors. This is only true 
for savepoints, and checkpoints actually store the state descrtiptors, see 
[MetadataV2V3SerializerBase 
|https://github.com/apache/flink/blob/2b1a9dea74a334adb1fe890f024f4153ad11a985/flink-runtime/src/main/java/org/apache/flink/runtime/checkpoint/metadata/MetadataV2V3SerializerBase.java#L318].
 BTW, even in current status, state-processor-API could still work well to read 
and modify the savepoints. It seems you have some additional requests, could 
you share what feature you're implementing? Or why you must need the meta 
stored in savepoint meta?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-21 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

Exactly, it can improve state-processor-api. I think it can also help the 
checkpoint meta more useful.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-21 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] THX for your reply.  We use state-processor-api to transform state 
now, but we find that we can not even build the state desc in any way. It seems 
like not friendly.
 # we can only get the typeSerializers or something that can not help we build 
a state desc (AggregatingStateDescriptor need the aggFunction).
 # even we just get the above, we can not get it easily, cause most of these 
are stored in the state file.

I think now the checkpoint meta do not have the power of self-described.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-21 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

[~liufangqi] Any changes to checkpoint persistent content requires attention to 
consider backward compatibility. I think your request sounds more like related 
to state-processor-API, could you give details on what current 
state-processor-API cannot do to meet your request? Moreover, do you think we 
can just change on state-processor-API to meet your request?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-20 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] , Actually, we can restore StateMetaInfoSnapshot in the state file, 
but in different mode of state snapshot, there are some diff :
 # 
org.apache.flink.contrib.streaming.state.snapshot.RocksIncrementalSnapshotStrategy:
 we use a different output stream from state output stream to write meta.
 # org.apache.flink.contrib.streaming.state.snapshot.RocksFullSnapshotStrategy 
/ org.apache.flink.runtime.state.DefaultOperatorStateBackendSnapshotStrategy : 
we use one same output stream to write both state data and meta data which 
means that we will load the StateMetaInfoSnapshot hardly if the state is so big.

So can we just use different output stream to write the meta data and store the 
StreamHandle in a different way like RocksIncrementalSnapshotStrategy. I think 
it can improve the performance of restoring state.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~mayuehappy] surely, when flink job start or recovery, the info is not 
necessary. But when we use state-processor-api, maybe it's necessary to get 
relevant info lightly. Maybe the checkpoint meta is not only used for recovery 
only anymore, we need it everywhere when we try to process state.
Or we can place the state desc in other file maybe? 

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Yue Ma (Jira)


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

Yue Ma commented on FLINK-25360:


 
[~liufangqi] i think adding this information to the metadata still requires a 
certain amount of change, it is also not necessary for job start or recovery. 
So I think whether it is worth doing this thing needs to be discussed.
 
 

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
> Attachments: image-2021-12-17-20-01-42-423.png
>
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~mayuehappy] THX for your reply, you are right. If the info is store in the 
state file only that we can not get this simply as the state file may be so 
huge. In most case, we want use this info locally. CheckpointMetadata is a tiny 
file, and we can deserialize it simply.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Yue Ma (Jira)


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

Yue Ma commented on FLINK-25360:


[~yunta] The StateSerializer would be persisted in the StateMetaInfoSnapshot in 
the state file. I think [~liufangqi]  means he wants to get this information 
from the checkpoint metadata instead of downloading these files.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Jira


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

刘方奇 commented on FLINK-25360:
-

[~yunta] Hi, do you mean that CheckpointMetadata.class contain these info, I 
didn't find in this.

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)


[jira] [Commented] (FLINK-25360) Add State Desc to CheckpointMetadata

2021-12-17 Thread Yun Tang (Jira)


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

Yun Tang commented on FLINK-25360:
--

The state name and state serializer have been persisted, what else do you want?

> Add State Desc to CheckpointMetadata
> 
>
> Key: FLINK-25360
> URL: https://issues.apache.org/jira/browse/FLINK-25360
> Project: Flink
>  Issue Type: Improvement
>  Components: Runtime / Checkpointing, Runtime / State Backends
>Reporter: 刘方奇
>Priority: Major
>
> Now we can't get the State Descriptor info in the checkpoint meta. Like the 
> case if we use state-processor-api to load state then rewrite state, we can't 
> flexible use the state. 
> Maybe there are other cases we need the State Descriptor, so can we add this 
> info?



--
This message was sent by Atlassian Jira
(v8.20.1#820001)