[
https://issues.apache.org/jira/browse/FLINK-37122?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
]
ChaomingZhang updated FLINK-37122:
----------------------------------
Component/s: Flink CDC
> NoSuchMethodError occurs when using flink 1.18.1.
> -------------------------------------------------
>
> Key: FLINK-37122
> URL: https://issues.apache.org/jira/browse/FLINK-37122
> Project: Flink
> Issue Type: Bug
> Components: Flink CDC
> Reporter: ChaomingZhang
> Priority: Major
>
> Since 1.19 all metadata about the task should be provided uniformly by
> getTaskInfo(). See more details in
> [FLIP-382|https://cwiki.apache.org/confluence/display/FLINK/FLIP-382%3A+Unify+the+Provision+of+Diverse+Metadata+for+Context-like+APIs].
> {code:java}
> // SchemaOperator.java
> @Override
> public void open() throws Exception {
> super.open();
> // java.lang.NoSuchMethodError
> subTaskId = getRuntimeContext().getTaskInfo().getIndexOfThisSubtask();
> upstreamSchemaTable = HashBasedTable.create();
> ...
> } {code}
> Some operators use this new methods which can ONLY be found after 1.19, which
> causes a NoSuchMethodError at runtime.
> {code:java}
> 2025-01-14 18:21:11,433 INFO [flink-akka.actor.default-dispatcher-4]
> org.apache.flink.runtime.executiongraph.ExecutionGraph [] - Source:
> MySQL Source -> SchemaOperator -> PrePartition (2/2)
> (a162db8a1bcd710c1444c2ea08bee4b3_717c7b8afebbfb7137f6f0f99beb2a94_1_0)
> switched from INITIALIZING to FAILED on antc4flink99035534-taskmanager-1-2 @
> 100.83.95.196 (dataPort=44721).
> java.lang.NoSuchMethodError: 'org.apache.flink.api.common.TaskInfo
> org.apache.flink.streaming.api.operators.StreamingRuntimeContext.getTaskInfo()'
> at
> org.apache.flink.cdc.runtime.operators.schema.regular.SchemaOperator.open(SchemaOperator.java:136)
> ~[?:?]
> at
> org.apache.flink.streaming.runtime.tasks.RegularOperatorChain.initializeStateAndOpenOperators(RegularOperatorChain.java:107)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreGates(StreamTask.java:817)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.streaming.runtime.tasks.StreamTaskActionExecutor$1.call(StreamTaskActionExecutor.java:55)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.restoreInternal(StreamTask.java:792)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.streaming.runtime.tasks.StreamTask.restore(StreamTask.java:757)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:968)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at
> org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:937)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:761)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:577)
> ~[flink-dist_2.12-1.18.1-SNAPSHOT.jar:1.18.1-SNAPSHOT]
> at java.lang.Thread.run(Thread.java:991) ~[?:?]
> {code}
> And this bug can be easily reproduced by running quick start examples
> [mysql-to-doris.|https://nightlies.apache.org/flink/flink-cdc-docs-release-3.2/docs/get-started/quickstart/mysql-to-doris/]
>
--
This message was sent by Atlassian Jira
(v8.20.10#820010)