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

Jiangjie Qin commented on FLINK-11654:
--------------------------------------

In practice, typically Kafka transaction.ids are assigned in the following way:
 # Users specifies something for each application to define its 
_transactional.id space,_ this guarantees no conflict _transactional.ids_ 
between applications.
 # Each application assigns producers with _transactional.id_ in their own 
space.

For Flink, JID can not be used because it may change across two runs of the 
same job. However, it seems {{JobName}} might be a reasonable option because it 
should probably be unique for each Job / Application, and it's not supposed to 
change. Changing the job name effectively make it another application and the 
exactly once guarantee for the previous application is no longer applicable.

[~jkreileder] the case you saw producers get fenced in a single job might be 
caused by FLINK-10455.

> Multiple transactional KafkaProducers writing to same cluster have clashing 
> transaction IDs
> -------------------------------------------------------------------------------------------
>
>                 Key: FLINK-11654
>                 URL: https://issues.apache.org/jira/browse/FLINK-11654
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Kafka
>    Affects Versions: 1.7.1
>            Reporter: Jürgen Kreileder
>            Priority: Major
>             Fix For: 1.9.0
>
>
> We run multiple jobs on a cluster which write a lot to the same Kafka topic 
> from identically named sinks. When EXACTLY_ONCE semantic is enabled for the 
> KafkaProducers we run into a lot of ProducerFencedExceptions and all jobs go 
> into a restart cycle.
> Example exception from the Kafka log:
>  
> {code:java}
> [2019-02-18 18:05:28,485] ERROR [ReplicaManager broker=1] Error processing 
> append operation on partition finding-commands-dev-1-0 
> (kafka.server.ReplicaManager)
> org.apache.kafka.common.errors.ProducerFencedException: Producer's epoch is 
> no longer valid. There is probably another producer with a newer epoch. 483 
> (request epoch), 484 (server epoch)
> {code}
> The reason for this is the way FlinkKafkaProducer initializes the 
> TransactionalIdsGenerator:
> The IDs are only guaranteed to be unique for a single Job. But they can clash 
> between different Jobs (and Clusters).
>  
>  
> {code:java}
> --- 
> a/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
> +++ 
> b/flink-connectors/flink-connector-kafka/src/main/java/org/apache/flink/streaming/connectors/kafka/FlinkKafkaProducer.java
> @@ -819,6 +819,7 @@ public class FlinkKafkaProducer<IN>
>                 nextTransactionalIdHintState = 
> context.getOperatorStateStore().getUnionListState(
>                         NEXT_TRANSACTIONAL_ID_HINT_DESCRIPTOR);
>                 transactionalIdsGenerator = new TransactionalIdsGenerator(
> + // the prefix probably should include job id and maybe cluster id
>                         getRuntimeContext().getTaskName() + "-" + 
> ((StreamingRuntimeContext) getRuntimeContext()).getOperatorUniqueID(),
>                         getRuntimeContext().getIndexOfThisSubtask(),
>                         
> getRuntimeContext().getNumberOfParallelSubtasks(),{code}
>  
>  



--
This message was sent by Atlassian JIRA
(v7.6.14#76016)

Reply via email to