Hi Min,

The uid is used to matching the operator state stored in the 
checkpoint/savepoint to an operator[1]. So you only need to specify the uid for 
stateful operators.
1) If you have not specified the uid for an operator, it will generate a uid 
for it in a deterministic way[2] for it. The generated uid doesn't change for 
the same job.
2) However, it's encouraged to set uid for stateful operators to allow for job 
evolution. The dynamically generated uid is not guaranteed to remain the same 
if the job has changed, i.e. adding/removing operators in the job graph. If you 
want to reuse state after job evolution, you need to set the uid explicitly.

So for the example you give, I think you don't need to specify the uid for the 
map and print operator.

[1] 
https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#matching-operator-state
 
<https://ci.apache.org/projects/flink/flink-docs-master/ops/upgrading.html#matching-operator-state>
[2] 
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>
> 在 2019年10月24日,下午11:22,min....@ubs.com 写道:
> 
> Hi,
>  
> I have some simple questions on the uid as well.
>  
> 1)      Do we add a uid for every operator e.g. print(), addSink and 
> addSource?
> 2)      For chained operators, do we need to uids for each operator? Or just 
> the last operator?
> e.g. .map(....).uid("some-id").print().uid("print-id");
>  
>  
> Regards,
>  
> Min
>  
> From: John Smith [mailto:java.dev....@gmail.com] 
> Sent: Donnerstag, 24. Oktober 2019 16:32
> To: Dian Fu
> Cc: user
> Subject: [External] Re: Does operator uid() have to be unique across all jobs?
>  
> Ok cool. Thanks
> 
> BTW this seems a bit cumbersome...
> 
> .map(....).uid("some-id").name("some-id");
>  
> On Wed, 23 Oct 2019 at 21:13, Dian Fu <dian0511...@gmail.com 
> <mailto:dian0511...@gmail.com>> wrote:
> Yes, you can use it in another job. The uid needs only to be unique within a 
> job.
> 
> > 在 2019年10月24日,上午5:42,John Smith <java.dev....@gmail.com 
> > <mailto:java.dev....@gmail.com>> 写道:
> > 
> > When setting uid() of an operator does it have to be unique across all jobs 
> > or just unique within a job?
> > 
> > For example can I use env.addSource(myKafkaConsumer).uid("kafka-consumer") 
> > in another job?
> 
> 
> E-mails can involve SUBSTANTIAL RISKS, e.g. lack of confidentiality, 
> potential manipulation of contents and/or sender's address, incorrect 
> recipient (misdirection), viruses etc. Based on previous e-mail 
> correspondence with you and/or an agreement reached with you, UBS considers 
> itself authorized to contact you via e-mail. UBS assumes no responsibility 
> for any loss or damage resulting from the use of e-mails. 
> The recipient is aware of and accepts the inherent risks of using e-mails, in 
> particular the risk that the banking relationship and confidential 
> information relating thereto are disclosed to third parties.
> UBS reserves the right to retain and monitor all messages. Messages are 
> protected and accessed only in legally justified cases.
> For information on how UBS uses and discloses personal data, how long we 
> retain it, how we keep it secure and your data protection rights, please see 
> our Privacy Notice http://www.ubs.com/privacy-statement 
> <http://www.ubs.com/privacy-statement>

Reply via email to