Flink CDC Issue Import created FLINK-34756: ----------------------------------------------
Summary: [MySQL-CDC] serverId parameter out of Integer range Key: FLINK-34756 URL: https://issues.apache.org/jira/browse/FLINK-34756 Project: Flink Issue Type: Bug Components: Flink CDC Reporter: Flink CDC Issue Import **Describe the bug(Please use English)** Now the 'server_id' is store as int, but the value of parameter 'server_id' would be larger than INT_MAX, this would cause IntegerParseException **Environment :** - Flink version : 1.16 - Flink CDC version: 2.3.0 - Database and version: mysql 5.7 **To Reproduce** Steps to reproduce the behavior: 1. The test data : MySQL instance in Aliyun Cloud, with a large server_id (out of INT range) ```bash mysql> show variables like '%server_id%'; +----------------+------------+ | Variable_name | Value | +----------------+------------+ | server_id | 2437466879 | | server_id_bits | 32 | +----------------+------------+ ``` 2. The test code : ```java public class MySqlSourceExample { public static void main(String[] args) throws Exception { MySqlSource<String> mySqlSource = MySqlSource.<String>builder() .hostname("host") .port(3306) .databaseList("dbs") // set captured database .tableList("tables") // set captured table .username("user") .password("password") .serverId("1-2437466879") .deserializer(new JsonDebeziumDeserializationSchema()) // converts SourceRecord to JSON String .build(); StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); env .fromSource(mySqlSource, WatermarkStrategy.noWatermarks(), "MySQL Source") // set 4 parallel source tasks .setParallelism(4) .print().setParallelism(1); // use parallelism 1 for sink to keep message ordering env.execute("Print MySQL Snapshot + Binlog"); } } ``` 3. The error : ```bash Exception in thread "main" java.lang.IllegalStateException: The server id 2437466879 is not a valid numeric. at com.ververica.cdc.connectors.mysql.source.config.ServerIdRange.parseServerId(ServerIdRange.java:108) at com.ververica.cdc.connectors.mysql.source.config.ServerIdRange.from(ServerIdRange.java:96) at com.ververica.cdc.connectors.mysql.source.config.MySqlSourceConfigFactory.serverId(MySqlSourceConfigFactory.java:130) at com.ververica.cdc.connectors.mysql.source.MySqlSourceBuilder.serverId(MySqlSourceBuilder.java:108) at com.bytedance.openplatform.flink.cdc.examples.MySqlSourceExample.main(MySqlSourceExample.java:25) Caused by: java.lang.NumberFormatException: For input string: "2437466879" at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65) at java.lang.Integer.parseInt(Integer.java:583) at java.lang.Integer.parseInt(Integer.java:615) at com.ververica.cdc.connectors.mysql.source.config.ServerIdRange.parseServerId(ServerIdRange.java:105) ... 4 more ``` **Additional Description** If applicable, add screenshots to help explain your problem. ---------------- Imported from GitHub ---------------- Url: https://github.com/apache/flink-cdc/issues/1770 Created by: [legendtkl|https://github.com/legendtkl] Labels: bug, Created at: Wed Nov 23 17:36:24 CST 2022 State: open -- This message was sent by Atlassian Jira (v8.20.10#820010)