You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "vinoyang (JIRA)" <ji...@apache.org> on 2019/06/26 03:33:00 UTC

[jira] [Comment Edited] (FLINK-12976) Bump Kafka client version to 2.3.0 for universal Kafka connector

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

vinoyang edited comment on FLINK-12976 at 6/26/19 3:32 AM:
-----------------------------------------------------------

I can make sure that there are compatibility issues when bumping Kafka client version from 2.2.0 to 2.3.0

log details: [https://api.travis-ci.org/v3/job/550192082/log.txt]

One of them is the {{FlinkKafkaInternalProducer}} uses reflection to get the direct inner {{nextSequence}} field of {{TransactionManager}}.

Before 2.3.0, the type of this field is:
{code:java}
private final Map<TopicPartition, Integer> nextSequence;
{code}
How in 2.3.0, the {{TransactionManager}} changed its implementation, it introduced two inner classes, named {{TopicPartitionBookkeeper}} and {{TopicPartitionEntry}}. And the {{nextSequence}} is put into {{TopicPartitionEntry}}. 
{code:java}
Caused by: java.lang.RuntimeException: Incompatible KafkaProducer version
	at org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaInternalProducer.getValue(FlinkKafkaInternalProducer.java:262)
	at org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaInternalProducer.getValue(FlinkKafkaInternalProducer.java:253)
	at org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaInternalProducer.resumeTransaction(FlinkKafkaInternalProducer.java:156)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.recoverAndCommit(FlinkKafkaProducer.java:752)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.recoverAndCommit(FlinkKafkaProducer.java:98)
	at org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.recoverAndCommitInternal(TwoPhaseCommitSinkFunction.java:393)
	at org.apache.flink.streaming.api.functions.sink.TwoPhaseCommitSinkFunction.initializeState(TwoPhaseCommitSinkFunction.java:351)
	at org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer.initializeState(FlinkKafkaProducer.java:870)
	at org.apache.flink.streaming.util.functions.StreamingFunctionUtils.tryRestoreFunction(StreamingFunctionUtils.java:178)
	at org.apache.flink.streaming.util.functions.StreamingFunctionUtils.restoreFunctionState(StreamingFunctionUtils.java:160)
	at org.apache.flink.streaming.api.operators.AbstractUdfStreamOperator.initializeState(AbstractUdfStreamOperator.java:96)
	at org.apache.flink.streaming.api.operators.AbstractStreamOperator.initializeState(AbstractStreamOperator.java:278)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.initializeState(StreamTask.java:856)
	at org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:365)
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:685)
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:515)
	at java.lang.Thread.run(Thread.java:748)
Caused by: java.lang.NoSuchFieldException: nextSequence
	at java.lang.Class.getDeclaredField(Class.java:2070)
	at org.apache.flink.streaming.connectors.kafka.internal.FlinkKafkaInternalProducer.getValue(FlinkKafkaInternalProducer.java:258)
	... 16 more
{code}
So we need to change the way of getting this field. I will try to figure it out. cc [~aljoscha] and [~pnowojski].


was (Author: yanghua):
I can make sure that there are compatibility issues when bumping Kafka client version from 2.2.0 to 2.3.0

log details: [https://api.travis-ci.org/v3/job/550192082/log.txt]

One of them is the {{FlinkKafkaInternalProducer}} uses reflection to get the direct inner {{nextSequence}} field of {{TransactionManager}}.

Before 2.3.0, the type of this field is:
{code:java}
private final Map<TopicPartition, Integer> nextSequence;
{code}
How in 2.3.0, the {{TransactionManager}} changed its implementation, it introduced two inner classes, named {{TopicPartitionBookkeeper}} and {{TopicPartitionEntry}}. And the {{nextSequence}} is put into {{TopicPartitionEntry}}. 

So we need to change the way of getting this field. I will try to figure it out. cc [~aljoscha] and [~pnowojski].

 

 

> Bump Kafka client version to 2.3.0 for universal Kafka connector
> ----------------------------------------------------------------
>
>                 Key: FLINK-12976
>                 URL: https://issues.apache.org/jira/browse/FLINK-12976
>             Project: Flink
>          Issue Type: Sub-task
>          Components: Connectors / Kafka
>            Reporter: vinoyang
>            Assignee: vinoyang
>            Priority: Major
>              Labels: pull-request-available
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
> Recently, Kafka 2.3.0 has released, see here: [https://www.apache.org/dist/kafka/2.3.0/RELEASE_NOTES.html]
> We'd better bump the dependency version of Kafka client to 2.3.0 to track the newest version.



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)