[ https://issues.apache.org/jira/browse/FLINK-9295?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16472055#comment-16472055 ]
ASF GitHub Bot commented on FLINK-9295: --------------------------------------- Github user StephanEwen commented on a diff in the pull request: https://github.com/apache/flink/pull/5977#discussion_r187640541 --- Diff: flink-core/src/main/java/org/apache/flink/api/common/functions/RuntimeContext.java --- @@ -71,6 +71,17 @@ @PublicEvolving MetricGroup getMetricGroup(); + /** + * Returned value is guaranteed to be unique between operators within the same job and to be + * stable and the same across job submissions. + * + * <p>This operation is currently only supported in Streaming (DataStream) contexts. + * + * @return String representation of the operator's unique id. + */ + @PublicEvolving + String getOperatorUniqueID(); --- End diff -- Rather than adding this here and failing for *DataSet* programs, hoe about adding this to `StreamingRuntimeContext` and casting inside the Kafka Producer? Not super pretty, but nicer than having something that looks like a pretty generic concept (operator id) throwing an exception in a whole class of programs (batch jobs). This problem should go away anyways with the batch / streaming unification later. > FlinkKafkaProducer011 sometimes throws ProducerFencedExceptions when in > EXACTLY_ONCE mode > ----------------------------------------------------------------------------------------- > > Key: FLINK-9295 > URL: https://issues.apache.org/jira/browse/FLINK-9295 > Project: Flink > Issue Type: Bug > Components: Kafka Connector > Affects Versions: 1.4.2 > Reporter: Christopher Ng > Assignee: Piotr Nowojski > Priority: Major > Fix For: 1.5.0 > > > {{FlinkKafkaProducer011}} can throw {{ProducerFencedExceptions}} if multiple > sinks are used within the same sub-task. This can happen when > operator-chaining results in two different sinks in the same topology being > chained into a task, and thus into each of its sub-tasks. > The problem is that {{TransactionIdsGenerator}} only takes into account the > task name, the subtask index, the number of subtasks, and a couple of other > things. All the attributes are the same between different > {{FlinkKafkaProducer011s}} within the same sub-task, so they get the same > transaction ids and one of them ends up failing. -- This message was sent by Atlassian JIRA (v7.6.3#76005)