[
https://issues.apache.org/jira/browse/FLINK-8306?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16316300#comment-16316300
]
ASF GitHub Bot commented on FLINK-8306:
---------------------------------------
Github user pnowojski commented on the issue:
https://github.com/apache/flink/pull/5200
Btw, doesn't the
`org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread#setOffsetsToCommit`
have a race condition on those lines:
```
// record the work to be committed by the main consumer thread
and make sure the consumer notices that
if (nextOffsetsToCommit.getAndSet(offsetsToCommit) != null) {
log.warn("Committing offsets to Kafka takes longer than
the checkpoint interval. " +
"Skipping commit of previous offsets
because newer complete checkpoint offsets are available. " +
"This does not compromise Flink's
checkpoint integrity.");
}
this.offsetCommitCallback = commitCallback;
```
(`getAndSet` followed by `volatile`) with
`org.apache.flink.streaming.connectors.kafka.internal.KafkaConsumerThread#run`?
However currently it doesn't seem to be so important, since:
1. it's used only for metrics
2. and only first `successfulCommits.inc()` can be omitted because of
that., because (at least now) `offsetCommitCallback` is a `write-once`
variable, and any subsequent overwrites, are overwriting it to the same value.
> FlinkKafkaConsumerBaseTest has invalid mocks on final methods
> -------------------------------------------------------------
>
> Key: FLINK-8306
> URL: https://issues.apache.org/jira/browse/FLINK-8306
> Project: Flink
> Issue Type: Bug
> Components: Kafka Connector, Tests
> Reporter: Tzu-Li (Gordon) Tai
> Assignee: Tzu-Li (Gordon) Tai
> Priority: Critical
> Fix For: 1.5.0, 1.4.1
>
>
> The {{FlinkKafkaConsumerBaseTest}} has invalid mocks on a final
> {{AbstractFetcher::commitInternalOffsetsToKafka(...)}} method. While an easy
> fix would be to simply make that method non-final, that is not ideal since it
> would be best that the method is left final to prevent overrides in
> subclasses.
> This suggests that offset committing functionality is too tightly coupled
> with the {{AbstractFetcher}}, making it hard to perform concise tests to
> verify offset committing.
> I suggest that we decouple record fetching and offset committing as separate
> services behind different interfaces. We should introduce a new interface,
> say {{KafkaOffsetCommitter}}, and test against that instead. Initially, we
> can simply let {{AbstractFetcher}} implement {{KafkaOffsetCommitter}}.
--
This message was sent by Atlassian JIRA
(v6.4.14#64029)