You are viewing a plain text version of this content. The canonical link for it is here.
Posted to user@flink.apache.org by M Singh <ma...@yahoo.com> on 2019/11/22 03:10:33 UTC

Apache Flink - Uid and name for Flink sources and sinks

Hi Folks:
I am assigning uid and name for all stateful processors in our application and wanted to find out the following:
1. Should we assign uid and name to the sources and sinks too ?  2. What are the pros and cons of adding uid to sources and sinks ?3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?  5. Can the sinks and sources without uid restart from savepoints ?6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
Thanks for your help.
Mans

Re: Apache Flink - Uid and name for Flink sources and sinks

Posted by M Singh <ma...@yahoo.com>.
 
Thanks DIan for your pointers.  Mans    On Sunday, November 24, 2019, 08:57:53 PM EST, Dian Fu <di...@gmail.com> wrote:  
 
 Hi Mans,
Please see my reply inline below.


在 2019年11月25日,上午5:42,M Singh <ma...@yahoo.com> 写道:
 Thanks Dian for your answers.
A few more questions:
1. If I do not assign uids to operators/sources and sinks - I am assuming the framework assigns it one.  Now how does another run of the the same application using the previous runs savepoint/checkpoint match it's tasks/operators to the savepoint/checkpoint state of the application ? 

You are right that the framework will generate an uid for an operator if it's not assigned. The uid is generated in a deterministic way to ensure that the uid for the same operator remains the same as previous runs(under certain conditions). 
The uid generation algorithm:https://github.com/apache/flink/blob/fd511c345eac31f03b801ff19dbf1f8c86aae760/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java#L78


2. Is the operatorID in the checkpoint state the same as uid ?  

3. Do you have any pointer as to how an operatorID is generated for the checkpoint and who can it be mapped to back to the operator for troubleshooting purposes ?

The OperatorID is constructed from the uid and they are the same:https://github.com/apache/flink/blob/66b979efc7786edb1a207339b8670d0e82c459a7/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java#L292


Regarding id attribute - I meant the following:
https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java#L139


However, I realized that this is not unique across applications runs and so not a good candidate.
Thanks again for your help.





    On Sunday, November 24, 2019, 04:55:55 AM EST, Dian Fu <di...@gmail.com> wrote:  
 
 1. Should we assign uid and name to the sources and sinks too ?  
>> If the sources/sinks have used state, you should assign uid for them. This is usually true for sources. 

2. What are the pros and cons of adding uid to sources and sinks ?
>> I'm not seeing the cons for assigning uid to sources and sinks. So I guess assigning the uids for sources/sinks is always a good practice.

3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?
>> Could you see if this could answer you question: https://stackoverflow.com/questions/46112142/apache-flink-set-operator-uid-vs-uidhash

4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?>> It depends on whether the sources/sinks uses state. I think most sources use state to maintaining the read offset.  5. Can the sinks and sources without uid restart from savepoints ?
>> The same as above.

6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
>> Not sure what do you mean by "attribute id". Could you give some more detailed information about it?

Regards,
Dian
On Fri, Nov 22, 2019 at 6:27 PM M Singh <ma...@yahoo.com> wrote:

 
Hi Folks - Please let me know if you have any advice on the best practices for setting uid for sources and sinks.  Thanks.  Mans    On Thursday, November 21, 2019, 10:10:49 PM EST, M Singh <ma...@yahoo.com> wrote:  
 
 Hi Folks:
I am assigning uid and name for all stateful processors in our application and wanted to find out the following:
1. Should we assign uid and name to the sources and sinks too ?  2. What are the pros and cons of adding uid to sources and sinks ?3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?  5. Can the sinks and sources without uid restart from savepoints ?6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
Thanks for your help.
Mans  
  

  

Re: Apache Flink - Uid and name for Flink sources and sinks

Posted by Dian Fu <di...@gmail.com>.
Hi Mans,

Please see my reply inline below.


> 在 2019年11月25日,上午5:42,M Singh <ma...@yahoo.com> 写道:
> 
> Thanks Dian for your answers.
> 
> A few more questions:
> 
> 1. If I do not assign uids to operators/sources and sinks - I am assuming the framework assigns it one.  Now how does another run of the the same application using the previous runs savepoint/checkpoint match it's tasks/operators to the savepoint/checkpoint state of the application ? 

You are right that the framework will generate an uid for an operator if it's not assigned. The uid is generated in a deterministic way to ensure that the uid for the same operator remains the same as previous runs(under certain conditions). 

The uid generation algorithm:
https://github.com/apache/flink/blob/fd511c345eac31f03b801ff19dbf1f8c86aae760/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java#L78 <https://github.com/apache/flink/blob/fd511c345eac31f03b801ff19dbf1f8c86aae760/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamGraphHasherV2.java#L78>


> 2. Is the operatorID in the checkpoint state the same as uid ?  
> 3. Do you have any pointer as to how an operatorID is generated for the checkpoint and who can it be mapped to back to the operator for troubleshooting purposes ?

The OperatorID is constructed from the uid and they are the same:
https://github.com/apache/flink/blob/66b979efc7786edb1a207339b8670d0e82c459a7/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java#L292 <https://github.com/apache/flink/blob/66b979efc7786edb1a207339b8670d0e82c459a7/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/graph/StreamingJobGraphGenerator.java#L292>
> 
> Regarding id attribute - I meant the following:
> 
> https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java#L139 <https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java#L139>
> However, I realized that this is not unique across applications runs and so not a good candidate.
> 
> Thanks again for your help.
> 
> 
> 
> 
> 
> On Sunday, November 24, 2019, 04:55:55 AM EST, Dian Fu <di...@gmail.com> wrote:
> 
> 
> 1. Should we assign uid and name to the sources and sinks too ?  
> >> If the sources/sinks have used state, you should assign uid for them. This is usually true for sources. 
> 
> 2. What are the pros and cons of adding uid to sources and sinks ?
> >> I'm not seeing the cons for assigning uid to sources and sinks. So I guess assigning the uids for sources/sinks is always a good practice.
> 
> 3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?
> >> Could you see if this could answer you question: https://stackoverflow.com/questions/46112142/apache-flink-set-operator-uid-vs-uidhash <https://stackoverflow.com/questions/46112142/apache-flink-set-operator-uid-vs-uidhash>
> 
> 4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?
> >> It depends on whether the sources/sinks uses state. I think most sources use state to maintaining the read offset.
>   
> 5. Can the sinks and sources without uid restart from savepoints ?
> >> The same as above.
> 
> 6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
> >> Not sure what do you mean by "attribute id". Could you give some more detailed information about it?
> 
> Regards,
> Dian
> 
> On Fri, Nov 22, 2019 at 6:27 PM M Singh <mans2singh@yahoo.com <ma...@yahoo.com>> wrote:
> 
> Hi Folks - Please let me know if you have any advice on the best practices for setting uid for sources and sinks.  Thanks.  Mans
> On Thursday, November 21, 2019, 10:10:49 PM EST, M Singh <mans2singh@yahoo.com <ma...@yahoo.com>> wrote:
> 
> 
> Hi Folks:
> 
> I am assigning uid and name for all stateful processors in our application and wanted to find out the following:
> 
> 1. Should we assign uid and name to the sources and sinks too ?  
> 2. What are the pros and cons of adding uid to sources and sinks ?
> 3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?
> 4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?  
> 5. Can the sinks and sources without uid restart from savepoints ?
> 6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
> 
> Thanks for your help.
> 
> Mans


Re: Apache Flink - Uid and name for Flink sources and sinks

Posted by M Singh <ma...@yahoo.com>.
 Thanks Dian for your answers.
A few more questions:
1. If I do not assign uids to operators/sources and sinks - I am assuming the framework assigns it one.  Now how does another run of the the same application using the previous runs savepoint/checkpoint match it's tasks/operators to the savepoint/checkpoint state of the application ? 2. Is the operatorID in the checkpoint state the same as uid ?  3. Do you have any pointer as to how an operatorID is generated for the checkpoint and who can it be mapped to back to the operator for troubleshooting purposes ?

Regarding id attribute - I meant the following:
https://github.com/apache/flink/blob/master/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/datastream/DataStream.java#L139
However, I realized that this is not unique across applications runs and so not a good candidate.
Thanks again for your help.





    On Sunday, November 24, 2019, 04:55:55 AM EST, Dian Fu <di...@gmail.com> wrote:  
 
 1. Should we assign uid and name to the sources and sinks too ?  
>> If the sources/sinks have used state, you should assign uid for them. This is usually true for sources. 

2. What are the pros and cons of adding uid to sources and sinks ?
>> I'm not seeing the cons for assigning uid to sources and sinks. So I guess assigning the uids for sources/sinks is always a good practice.

3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?
>> Could you see if this could answer you question: https://stackoverflow.com/questions/46112142/apache-flink-set-operator-uid-vs-uidhash

4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?>> It depends on whether the sources/sinks uses state. I think most sources use state to maintaining the read offset.  5. Can the sinks and sources without uid restart from savepoints ?
>> The same as above.

6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
>> Not sure what do you mean by "attribute id". Could you give some more detailed information about it?

Regards,
Dian
On Fri, Nov 22, 2019 at 6:27 PM M Singh <ma...@yahoo.com> wrote:

 
Hi Folks - Please let me know if you have any advice on the best practices for setting uid for sources and sinks.  Thanks.  Mans    On Thursday, November 21, 2019, 10:10:49 PM EST, M Singh <ma...@yahoo.com> wrote:  
 
 Hi Folks:
I am assigning uid and name for all stateful processors in our application and wanted to find out the following:
1. Should we assign uid and name to the sources and sinks too ?  2. What are the pros and cons of adding uid to sources and sinks ?3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?  5. Can the sinks and sources without uid restart from savepoints ?6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
Thanks for your help.
Mans  
  

Re: Apache Flink - Uid and name for Flink sources and sinks

Posted by Dian Fu <di...@gmail.com>.
1. Should we assign uid and name to the sources and sinks too ?
>> If the sources/sinks have used state, you should assign uid for them.
This is usually true for sources.

2. What are the pros and cons of adding uid to sources and sinks ?
>> I'm not seeing the cons for assigning uid to sources and sinks. So I
guess assigning the uids for sources/sinks is always a good practice.

3. The sinks have uid and hashUid - which is the preferred attribute to
use  for allowing job restarts ?
>> Could you see if this could answer you question:
https://stackoverflow.com/questions/46112142/apache-flink-set-operator-uid-vs-uidhash

4. If sink and sources uid are not provided in the application, can they
still maintain state across job restarts from checkpoints ?
>> It depends on whether the sources/sinks uses state. I think most sources
use state to maintaining the read offset.

5. Can the sinks and sources without uid restart from savepoints ?
>> The same as above.

6. The data streams have an attribute id -  How is this generated and can
this be used for creating a uid for the sink ?
>> Not sure what do you mean by "attribute id". Could you give some more
detailed information about it?

Regards,
Dian

On Fri, Nov 22, 2019 at 6:27 PM M Singh <ma...@yahoo.com> wrote:

>
> Hi Folks - Please let me know if you have any advice on the best practices
> for setting uid for sources and sinks.  Thanks.  Mans
> On Thursday, November 21, 2019, 10:10:49 PM EST, M Singh <
> mans2singh@yahoo.com> wrote:
>
>
> Hi Folks:
>
> I am assigning uid and name for all stateful processors in our application
> and wanted to find out the following:
>
> 1. Should we assign uid and name to the sources and sinks too ?
> 2. What are the pros and cons of adding uid to sources and sinks ?
> 3. The sinks have uid and hashUid - which is the preferred attribute to
> use  for allowing job restarts ?
> 4. If sink and sources uid are not provided in the application, can they
> still maintain state across job restarts from checkpoints ?
> 5. Can the sinks and sources without uid restart from savepoints ?
> 6. The data streams have an attribute id -  How is this generated and can
> this be used for creating a uid for the sink ?
>
> Thanks for your help.
>
> Mans
>

Re: Apache Flink - Uid and name for Flink sources and sinks

Posted by M Singh <ma...@yahoo.com>.
 
Hi Folks - Please let me know if you have any advice on the best practices for setting uid for sources and sinks.  Thanks.  Mans    On Thursday, November 21, 2019, 10:10:49 PM EST, M Singh <ma...@yahoo.com> wrote:  
 
 Hi Folks:
I am assigning uid and name for all stateful processors in our application and wanted to find out the following:
1. Should we assign uid and name to the sources and sinks too ?  2. What are the pros and cons of adding uid to sources and sinks ?3. The sinks have uid and hashUid - which is the preferred attribute to use  for allowing job restarts ?4. If sink and sources uid are not provided in the application, can they still maintain state across job restarts from checkpoints ?  5. Can the sinks and sources without uid restart from savepoints ?6. The data streams have an attribute id -  How is this generated and can this be used for creating a uid for the sink ?  
Thanks for your help.
Mans