You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by "Yun Tang (Jira)" <ji...@apache.org> on 2023/03/04 15:43:00 UTC

[jira] [Updated] (FLINK-31324) Broken SingleThreadFetcherManager constructor API

     [ https://issues.apache.org/jira/browse/FLINK-31324?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]

Yun Tang updated FLINK-31324:
-----------------------------
    Description: 
FLINK-28853 changed the default constructor of {{SingleThreadFetcherManager}}. Though the {{SingleThreadFetcherManager}} is annotated as {{Internal}}, it actually acts as some-degree public API, which is widely used in many connector projects:
[flink-cdc-connector|https://github.com/ververica/flink-cdc-connectors/blob/release-2.3.0/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java#L93], [flink-connector-mongodb|https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java#L58] and so on.

Once flink-1.17 is released, all these existing connectors are broken and cannot be used in new release version, and will throw exceptions like:

{code:java}
java.lang.NoSuchMethodError: org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager.<init>(Lorg/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue;Ljava/util/function/Supplier;)V
	at com.ververica.cdc.connectors.mysql.source.reader.MySqlSourceReader.<init>(MySqlSourceReader.java:91) ~[flink-sql-connector-mysql-cdc-2.3.0.jar:2.3.0]
	at com.ververica.cdc.connectors.mysql.source.MySqlSource.createReader(MySqlSource.java:159) ~[flink-sql-connector-mysql-cdc-2.3.0.jar:2.3.0]
	at org.apache.flink.streaming.api.operators.SourceOperator.initReader(SourceOperator.java:312) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask.init(SourceOperatorStreamTask.java:94) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:699) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:675) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:952) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:921) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:745) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
	at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_362]
{code}


Thus, I suggest to make the original SingleThreadFetcherManager constructor as depreacted instead of removing it.

  was:
FLINK-28853 changed the default constructor of {{SingleThreadFetcherManager}}. Though the {{SingleThreadFetcherManager}} is annotated as {{Internal}}, it actually acts as some-degree public API, which is widely used in many connector projects:
[flink-cdc-connector|https://github.com/ververica/flink-cdc-connectors/blob/release-2.3.0/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java#L93], [flink-connector-mongodb|https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java#L58] and so on.

Once flink-1.17 is released, all these existing connectors are broken and cannot be used in new release version. Thus, I suggest to make the original SingleThreadFetcherManager constructor as depreacted instead of removing it.


> Broken SingleThreadFetcherManager constructor API
> -------------------------------------------------
>
>                 Key: FLINK-31324
>                 URL: https://issues.apache.org/jira/browse/FLINK-31324
>             Project: Flink
>          Issue Type: Bug
>          Components: Connectors / Parent
>            Reporter: Yun Tang
>            Assignee: Yun Tang
>            Priority: Blocker
>             Fix For: 1.17.0
>
>
> FLINK-28853 changed the default constructor of {{SingleThreadFetcherManager}}. Though the {{SingleThreadFetcherManager}} is annotated as {{Internal}}, it actually acts as some-degree public API, which is widely used in many connector projects:
> [flink-cdc-connector|https://github.com/ververica/flink-cdc-connectors/blob/release-2.3.0/flink-connector-mysql-cdc/src/main/java/com/ververica/cdc/connectors/mysql/source/reader/MySqlSourceReader.java#L93], [flink-connector-mongodb|https://github.com/apache/flink-connector-mongodb/blob/main/flink-connector-mongodb/src/main/java/org/apache/flink/connector/mongodb/source/reader/MongoSourceReader.java#L58] and so on.
> Once flink-1.17 is released, all these existing connectors are broken and cannot be used in new release version, and will throw exceptions like:
> {code:java}
> java.lang.NoSuchMethodError: org.apache.flink.connector.base.source.reader.fetcher.SingleThreadFetcherManager.<init>(Lorg/apache/flink/connector/base/source/reader/synchronization/FutureCompletingBlockingQueue;Ljava/util/function/Supplier;)V
> 	at com.ververica.cdc.connectors.mysql.source.reader.MySqlSourceReader.<init>(MySqlSourceReader.java:91) ~[flink-sql-connector-mysql-cdc-2.3.0.jar:2.3.0]
> 	at com.ververica.cdc.connectors.mysql.source.MySqlSource.createReader(MySqlSource.java:159) ~[flink-sql-connector-mysql-cdc-2.3.0.jar:2.3.0]
> 	at org.apache.flink.streaming.api.operators.SourceOperator.initReader(SourceOperator.java:312) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.SourceOperatorStreamTask.init(SourceOperatorStreamTask.java:94) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:699) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:675) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:952) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:921) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:745) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562) ~[flink-dist-1.17-SNAPSHOT.jar:1.17-SNAPSHOT]
> 	at java.lang.Thread.run(Thread.java:750) ~[?:1.8.0_362]
> {code}
> Thus, I suggest to make the original SingleThreadFetcherManager constructor as depreacted instead of removing it.



--
This message was sent by Atlassian Jira
(v8.20.10#820010)