Hi Alexis, Sorry for the late answer ⊠got carried away with other tasks đ
I hope I get this right as there is a mixture of concepts in my mind with respect for the old and the savepoint API. Iâll try to answer for the new API. * If you want to patch an existing savepoint, you would load it into a SavepointWriter [1], this will basically copy the existing Savepoint, you then: * can remove or/and add state for operators [2] [3] * then you write the savepoint into the new location [4] * the old API had the restriction, that you had to change at least one operator state in order to be able to write the savepoint out to a new location. I donât believe this restriction applies for the new API * If you want to patch an existing state (e.g. for changing an incompatible schema), * you need to load/bootstrap this state by means of SavepointReader [5] and some of the readXXXState functions * then remove the existing state from previous savepoint [2] and add the new state [3] with the bootstrap transformation obtained above * if you want a state to be empty it suffices to [2] remove the existing state, if existed I hope this helps, ⊠happy to hear someone correct me, if mistaken Sincere regards Thias [1] org.apache.flink.state.api.SavepointWriter#fromExistingSavepoint(java.lang.String) [2] org.apache.flink.state.api.SavepointWriter#removeOperator(org.apache.flink.state.api.OperatorIdentifier) [3] org.apache.flink.state.api.SavepointWriter#withOperator(org.apache.flink.state.api.OperatorIdentifier, org.apache.flink.state.api.StateBootstrapTransformation<T>) [4] org.apache.flink.state.api.SavepointWriter#write [5] org.apache.flink.state.api.SavepointReader#read(org.apache.flink.streaming.api.environment.StreamExecutionEnvironment, java.lang.String, org.apache.flink.runtime.state.StateBackend) From: Alexis Sarda-Espinosa <sarda.espin...@gmail.com> Sent: Friday, October 27, 2023 4:29 PM To: Schwalbe Matthias <matthias.schwa...@viseca.ch> Cc: user <user@flink.apache.org> Subject: Re: Updating existing state with state processor API â EXTERNAL MESSAGE â CAUTION: Think Before You Click â Hi Matthias, Thanks for the response. I guess the specific question would be, if I work with an existing savepoint and pass an empty DataStream to OperatorTransformation#bootstrapWith, will the new savepoint end up with an empty state for the modified operator, or will it maintain the existing state because nothing was changed? Regards, Alexis. Am Fr., 27. Okt. 2023 um 08:40 Uhr schrieb Schwalbe Matthias <matthias.schwa...@viseca.ch<mailto:matthias.schwa...@viseca.ch>>: Good morning Alexis, Something like this we do all the time. Read and existing savepoint, copy some of the not to be changed operator states (keyed/non-keyed) over, and process/patch the remaining ones by transforming and bootstrapping to new state. I could spare more details for more specific questions, if you like đ Regards Thias PS: Iâm currently working on this ticket in order to get some glitches removed: FLINK-26585<https://issues.apache.org/jira/browse/FLINK-26585> From: Alexis Sarda-Espinosa <sarda.espin...@gmail.com<mailto:sarda.espin...@gmail.com>> Sent: Thursday, October 26, 2023 4:01 PM To: user <user@flink.apache.org<mailto:user@flink.apache.org>> Subject: Updating existing state with state processor API Hello, The documentation of the state processor API has some examples to modify an existing savepoint by defining a StateBootstrapTransformation. In all cases, the entrypoint is OperatorTransformation#bootstrapWith, which expects a DataStream. If I pass an empty DataStream to bootstrapWith and then apply the resulting transformation to an existing savepoint, will the transformation still receive data from the existing state? If the aforementioned is incorrect, I imagine I could instantiate a SavepointReader and create a DataStream of the existing state with it, which I could then pass to the bootstrapWith method directly or after "unioning" it with additional state. Would this work? Regards, Alexis. Diese Nachricht ist ausschliesslich fĂŒr den Adressaten bestimmt und beinhaltet unter UmstĂ€nden vertrauliche Mitteilungen. Da die Vertraulichkeit von e-Mail-Nachrichten nicht gewĂ€hrleistet werden kann, ĂŒbernehmen wir keine Haftung fĂŒr die GewĂ€hrung der Vertraulichkeit und Unversehrtheit dieser Mitteilung. Bei irrtĂŒmlicher Zustellung bitten wir Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie eventueller AnhĂ€nge. Jegliche unberechtigte Verwendung oder Verbreitung dieser Informationen ist streng verboten. This message is intended only for the named recipient and may contain confidential or privileged information. As the confidentiality of email communication cannot be guaranteed, we do not accept any responsibility for the confidentiality and the intactness of this message. If you have received it in error, please advise the sender by return e-mail and delete this message and any attachments. Any unauthorised use or dissemination of this information is strictly prohibited. Diese Nachricht ist ausschliesslich fĂŒr den Adressaten bestimmt und beinhaltet unter UmstĂ€nden vertrauliche Mitteilungen. Da die Vertraulichkeit von e-Mail-Nachrichten nicht gewĂ€hrleistet werden kann, ĂŒbernehmen wir keine Haftung fĂŒr die GewĂ€hrung der Vertraulichkeit und Unversehrtheit dieser Mitteilung. Bei irrtĂŒmlicher Zustellung bitten wir Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie eventueller AnhĂ€nge. Jegliche unberechtigte Verwendung oder Verbreitung dieser Informationen ist streng verboten. This message is intended only for the named recipient and may contain confidential or privileged information. As the confidentiality of email communication cannot be guaranteed, we do not accept any responsibility for the confidentiality and the intactness of this message. If you have received it in error, please advise the sender by return e-mail and delete this message and any attachments. Any unauthorised use or dissemination of this information is strictly prohibited.