RE: Loading broadcast state on BroadcastProcessFunction instantiation or open method

2022-09-28 Thread alfredo.vasquez.spglobal.com via user
Thank you, I have tried both approaches, Overriding open method did not work 
but by implementing CheckpointedFunction and overriding initializeState I was 
able to access and operate over broadcast state

@Override
public void initializeState(FunctionInitializationContext context) throws 
Exception {
BroadcastState state = 
context.getOperatorStateStore().getBroadcastState(RULES_DESCRIPTOR);
}

Regards,
From: Schwalbe Matthias 
Sent: Tuesday, September 27, 2022 8:12 AM
To: David Anderson ; Vasquez, Alfredo 

Cc: user@flink.apache.org
Subject: RE: Loading broadcast state on BroadcastProcessFunction instantiation 
or open method

EXTERNAL MESSAGE


Hi Alfredo,

Did you consider implementing 
org.apache.flink.streaming.api.checkpoint.CheckpointedFunction interface in 
your broadcast function … the initializeState(…) function should give you 
access to the state backend.

Kind regards

Thias


From: David Anderson mailto:dander...@apache.org>>
Sent: Tuesday, September 27, 2022 12:26 PM
To: alfredo.vasq...@spglobal.com<mailto:alfredo.vasq...@spglobal.com>
Cc: user@flink.apache.org<mailto:user@flink.apache.org>
Subject: Re: Loading broadcast state on BroadcastProcessFunction instantiation 
or open method

⚠EXTERNAL MESSAGE – CAUTION: Think Before You Click ⚠

Logically it would make sense to be able to initialize BroadcastState in the 
open method of a BroadcastProcessFunction, but in practice I don't believe it 
can be done -- because the necessary Context isn't made available.

Perhaps you could use the State Processor API to bootstrap some state into the 
broadcast state.

David

On Mon, Sep 26, 2022 at 6:07 PM 
alfredo.vasquez.spglobal.com<http://alfredo.vasquez.spglobal.com> via user 
mailto:user@flink.apache.org>> wrote:
Hello community.

Currently we have a BroadcastProcessFunction implementation that is storing the 
broadcast state using a MapStateDescriptor.
I have a use case that needs to load the BroadcastState to perform some 
operation before receiving any processElement or processBroadcastElement 
message.

Is there a way to load the BroadcastState on BroadcastProcessFunction  
instantiation, overriding open(Configuration parameters) method or by 
overriding some other callback function?

Kind regards,



The information contained in this message is intended only for the recipient, 
and may be a confidential attorney-client communication or may otherwise be 
privileged and confidential and protected from disclosure. If the reader of 
this message is not the intended recipient, or an employee or agent responsible 
for delivering this message to the intended recipient, please be aware that any 
dissemination or copying of this communication is strictly prohibited. If you 
have received this communication in error, please immediately notify us by 
replying to the message and deleting it from your computer. S Global Inc. 
reserves the right, subject to applicable local law, to monitor, review and 
process the content of any electronic message or information sent to or from 
S Global Inc. e-mail addresses without informing the sender or recipient of 
the message. By sending electronic message or information to S Global Inc. 
e-mail addresses you, as the sender, are consenting to S Global Inc. 
processing any of your personal data therein.
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.


RE: Loading broadcast state on BroadcastProcessFunction instantiation or open method

2022-09-27 Thread Schwalbe Matthias
Hi Alfredo,

Did you consider implementing 
org.apache.flink.streaming.api.checkpoint.CheckpointedFunction interface in 
your broadcast function … the initializeState(…) function should give you 
access to the state backend.

Kind regards

Thias


From: David Anderson 
Sent: Tuesday, September 27, 2022 12:26 PM
To: alfredo.vasq...@spglobal.com
Cc: user@flink.apache.org
Subject: Re: Loading broadcast state on BroadcastProcessFunction instantiation 
or open method

⚠EXTERNAL MESSAGE – CAUTION: Think Before You Click ⚠


Logically it would make sense to be able to initialize BroadcastState in the 
open method of a BroadcastProcessFunction, but in practice I don't believe it 
can be done -- because the necessary Context isn't made available.

Perhaps you could use the State Processor API to bootstrap some state into the 
broadcast state.

David

On Mon, Sep 26, 2022 at 6:07 PM 
alfredo.vasquez.spglobal.com<http://alfredo.vasquez.spglobal.com> via user 
mailto:user@flink.apache.org>> wrote:
Hello community.

Currently we have a BroadcastProcessFunction implementation that is storing the 
broadcast state using a MapStateDescriptor.
I have a use case that needs to load the BroadcastState to perform some 
operation before receiving any processElement or processBroadcastElement 
message.

Is there a way to load the BroadcastState on BroadcastProcessFunction  
instantiation, overriding open(Configuration parameters) method or by 
overriding some other callback function?

Kind regards,



The information contained in this message is intended only for the recipient, 
and may be a confidential attorney-client communication or may otherwise be 
privileged and confidential and protected from disclosure. If the reader of 
this message is not the intended recipient, or an employee or agent responsible 
for delivering this message to the intended recipient, please be aware that any 
dissemination or copying of this communication is strictly prohibited. If you 
have received this communication in error, please immediately notify us by 
replying to the message and deleting it from your computer. S Global Inc. 
reserves the right, subject to applicable local law, to monitor, review and 
process the content of any electronic message or information sent to or from 
S Global Inc. e-mail addresses without informing the sender or recipient of 
the message. By sending electronic message or information to S Global Inc. 
e-mail addresses you, as the sender, are consenting to S Global Inc. 
processing any of your personal data therein.
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.


Re: Loading broadcast state on BroadcastProcessFunction instantiation or open method

2022-09-27 Thread David Anderson
Logically it would make sense to be able to initialize BroadcastState in
the open method of a BroadcastProcessFunction, but in practice I don't
believe it can be done -- because the necessary Context isn't made
available.

Perhaps you could use the State Processor API to bootstrap some state into
the broadcast state.

David

On Mon, Sep 26, 2022 at 6:07 PM alfredo.vasquez.spglobal.com via user <
user@flink.apache.org> wrote:

> Hello community.
>
>
>
> Currently we have a BroadcastProcessFunction implementation that is
> storing the broadcast state using a MapStateDescriptor.
>
> I have a use case that needs to load the BroadcastState to perform some
> operation before receiving any processElement or processBroadcastElement
> message.
>
>
>
> Is there a way to load the BroadcastState on BroadcastProcessFunction
>  instantiation, overriding open(Configuration parameters) method or by
> overriding some other callback function?
>
>
>
> Kind regards,
>
> --
>
> The information contained in this message is intended only for the
> recipient, and may be a confidential attorney-client communication or may
> otherwise be privileged and confidential and protected from disclosure. If
> the reader of this message is not the intended recipient, or an employee or
> agent responsible for delivering this message to the intended recipient,
> please be aware that any dissemination or copying of this communication is
> strictly prohibited. If you have received this communication in error,
> please immediately notify us by replying to the message and deleting it
> from your computer. S Global Inc. reserves the right, subject to
> applicable local law, to monitor, review and process the content of any
> electronic message or information sent to or from S Global Inc. e-mail
> addresses without informing the sender or recipient of the message. By
> sending electronic message or information to S Global Inc. e-mail
> addresses you, as the sender, are consenting to S Global Inc. processing
> any of your personal data therein.
>


Loading broadcast state on BroadcastProcessFunction instantiation or open method

2022-09-26 Thread alfredo.vasquez.spglobal.com via user
Hello community.

Currently we have a BroadcastProcessFunction implementation that is storing the 
broadcast state using a MapStateDescriptor.
I have a use case that needs to load the BroadcastState to perform some 
operation before receiving any processElement or processBroadcastElement 
message.

Is there a way to load the BroadcastState on BroadcastProcessFunction  
instantiation, overriding open(Configuration parameters) method or by 
overriding some other callback function?

Kind regards,



The information contained in this message is intended only for the recipient, 
and may be a confidential attorney-client communication or may otherwise be 
privileged and confidential and protected from disclosure. If the reader of 
this message is not the intended recipient, or an employee or agent responsible 
for delivering this message to the intended recipient, please be aware that any 
dissemination or copying of this communication is strictly prohibited. If you 
have received this communication in error, please immediately notify us by 
replying to the message and deleting it from your computer. S Global Inc. 
reserves the right, subject to applicable local law, to monitor, review and 
process the content of any electronic message or information sent to or from 
S Global Inc. e-mail addresses without informing the sender or recipient of 
the message. By sending electronic message or information to S Global Inc. 
e-mail addresses you, as the sender, are consenting to S Global Inc. 
processing any of your personal data therein.