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

Thorne updated FLINK-38021:
---------------------------
    Description: 
The problem of double quotation marks in the default value of automatic table 
parsing

yaml config
{code:java}
source:  
  scan.binlog.newly-added-table.enabled: true{code}
 

 
{code:java}
CREATE TABLE `ac` (
  `id` int(11) NOT NULL,
  `ac_code` int(10) DEFAULT "0",
  `name` varchar(255) DEFAULT NULL,
  PRIMARY KEY (`id`)
) ENGINE=InnoDB ;SET FOREIGN_KEY_CHECKS = 1;
show create table ac ;CREATE TABLE `ac` (
  `id` int(11) NOT NULL,
  `ac_code` int(10) DEFAULT '0',
  `name` varchar(255) DEFAULT NULL,
  PRIMARY KEY (`id`)
) ENGINE=InnoDB DEFAULT CHARSET=latin1 {code}
 

error log
{code:java}
Caused by: java.lang.NumberFormatException: For input string: ""0""    at 
java.base/jdk.internal.math.FloatingDecimal.readJavaFormatString(FloatingDecimal.java:2054)
    at 
java.base/jdk.internal.math.FloatingDecimal.parseDouble(FloatingDecimal.java:110)
    at java.base/java.lang.Double.parseDouble(Double.java:651)    at 
io.debezium.connector.mysql.MySqlValueConverters.convertInteger(MySqlValueConverters.java:469)
    at 
io.debezium.jdbc.JdbcValueConverters.lambda$converter$5(JdbcValueConverters.java:295)
    at 
io.debezium.connector.mysql.MySqlDefaultValueConverter.parseDefaultValue(MySqlDefaultValueConverter.java:140)
    at 
io.debezium.relational.TableSchemaBuilder.lambda$addField$9(TableSchemaBuilder.java:393)
    at java.base/java.util.Optional.flatMap(Optional.java:289)    at 
io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:393) 
   at 
io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:149)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
    at 
java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:179)
    at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
    at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509) 
   at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
    at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
   at 
java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
    at 
io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:147)   
 at 
io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135)
    at 
io.debezium.connector.mysql.MySqlDatabaseSchema.lambda$applySchemaChange$2(MySqlDatabaseSchema.java:179)
    at java.base/java.lang.Iterable.forEach(Iterable.java:75)    at 
io.debezium.connector.mysql.MySqlDatabaseSchema.applySchemaChange(MySqlDatabaseSchema.java:179)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.EventDispatcherImpl$SchemaChangeEventReceiver.schemaChangeEvent(EventDispatcherImpl.java:314)
    at 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$handleQueryEvent$2(MySqlStreamingChangeEventSource.java:673)
    ... 12 more
 {code}
!image-2025-06-27-12-31-26-011.png!

As we can see, The mysql has cover it successfully ,but flink-cdc cannot cover 
it `desc table `

  was:
The problem of double quotation marks in the default value of automatic table 
parsing

yaml config
{code:java}
source:  
  scan.binlog.newly-added-table.enabled: true{code}
 

 
{code:java}
CREATE TABLE `ac` (
  `id` int(11) NOT NULL,
  `ac_code` int(10) DEFAULT "0",
  `name` varchar(255) DEFAULT NULL,
  PRIMARY KEY (`id`)
) ENGINE=InnoDB ;SET FOREIGN_KEY_CHECKS = 1;
show create table ac ;CREATE TABLE `ac` (
  `id` int(11) NOT NULL,
  `ac_code` int(10) DEFAULT '0',
  `name` varchar(255) DEFAULT NULL,
  PRIMARY KEY (`id`)
) ENGINE=InnoDB DEFAULT CHARSET=latin1 {code}
 

error log
{code:java}
Caused by: java.lang.NumberFormatException: For input string: ""0""    at 
java.base/jdk.internal.math.FloatingDecimal.readJavaFormatString(FloatingDecimal.java:2054)
    at 
java.base/jdk.internal.math.FloatingDecimal.parseDouble(FloatingDecimal.java:110)
    at java.base/java.lang.Double.parseDouble(Double.java:651)    at 
io.debezium.connector.mysql.MySqlValueConverters.convertInteger(MySqlValueConverters.java:469)
    at 
io.debezium.jdbc.JdbcValueConverters.lambda$converter$5(JdbcValueConverters.java:295)
    at 
io.debezium.connector.mysql.MySqlDefaultValueConverter.parseDefaultValue(MySqlDefaultValueConverter.java:140)
    at 
io.debezium.relational.TableSchemaBuilder.lambda$addField$9(TableSchemaBuilder.java:393)
    at java.base/java.util.Optional.flatMap(Optional.java:289)    at 
io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:393) 
   at 
io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:149)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
    at 
java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:179)
    at 
java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
    at 
java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509) 
   at 
java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
    at 
java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
    at 
java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234) 
   at 
java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
    at 
io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:147)   
 at 
io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135)
    at 
io.debezium.connector.mysql.MySqlDatabaseSchema.lambda$applySchemaChange$2(MySqlDatabaseSchema.java:179)
    at java.base/java.lang.Iterable.forEach(Iterable.java:75)    at 
io.debezium.connector.mysql.MySqlDatabaseSchema.applySchemaChange(MySqlDatabaseSchema.java:179)
    at 
org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.EventDispatcherImpl$SchemaChangeEventReceiver.schemaChangeEvent(EventDispatcherImpl.java:314)
    at 
io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$handleQueryEvent$2(MySqlStreamingChangeEventSource.java:673)
    ... 12 more
 {code}
 


> The problem of double quotation marks in the default value of automatic table 
> parsing
> -------------------------------------------------------------------------------------
>
>                 Key: FLINK-38021
>                 URL: https://issues.apache.org/jira/browse/FLINK-38021
>             Project: Flink
>          Issue Type: Bug
>          Components: Flink CDC
>    Affects Versions: cdc-3.4.0
>         Environment: flink:flink1.20.0
> flink-cdc :3.4.0
> mysql:5.7
> jdk:8
>            Reporter: Thorne
>            Priority: Major
>             Fix For: cdc-3.5.0
>
>         Attachments: image-2025-06-27-12-31-26-011.png
>
>
> The problem of double quotation marks in the default value of automatic table 
> parsing
> yaml config
> {code:java}
> source:  
>   scan.binlog.newly-added-table.enabled: true{code}
>  
>  
> {code:java}
> CREATE TABLE `ac` (
>   `id` int(11) NOT NULL,
>   `ac_code` int(10) DEFAULT "0",
>   `name` varchar(255) DEFAULT NULL,
>   PRIMARY KEY (`id`)
> ) ENGINE=InnoDB ;SET FOREIGN_KEY_CHECKS = 1;
> show create table ac ;CREATE TABLE `ac` (
>   `id` int(11) NOT NULL,
>   `ac_code` int(10) DEFAULT '0',
>   `name` varchar(255) DEFAULT NULL,
>   PRIMARY KEY (`id`)
> ) ENGINE=InnoDB DEFAULT CHARSET=latin1 {code}
>  
> error log
> {code:java}
> Caused by: java.lang.NumberFormatException: For input string: ""0""    at 
> java.base/jdk.internal.math.FloatingDecimal.readJavaFormatString(FloatingDecimal.java:2054)
>     at 
> java.base/jdk.internal.math.FloatingDecimal.parseDouble(FloatingDecimal.java:110)
>     at java.base/java.lang.Double.parseDouble(Double.java:651)    at 
> io.debezium.connector.mysql.MySqlValueConverters.convertInteger(MySqlValueConverters.java:469)
>     at 
> io.debezium.jdbc.JdbcValueConverters.lambda$converter$5(JdbcValueConverters.java:295)
>     at 
> io.debezium.connector.mysql.MySqlDefaultValueConverter.parseDefaultValue(MySqlDefaultValueConverter.java:140)
>     at 
> io.debezium.relational.TableSchemaBuilder.lambda$addField$9(TableSchemaBuilder.java:393)
>     at java.base/java.util.Optional.flatMap(Optional.java:289)    at 
> io.debezium.relational.TableSchemaBuilder.addField(TableSchemaBuilder.java:393)
>     at 
> io.debezium.relational.TableSchemaBuilder.lambda$create$2(TableSchemaBuilder.java:149)
>     at 
> java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.accept(ForEachOps.java:183)
>     at 
> java.base/java.util.stream.ReferencePipeline$2$1.accept(ReferencePipeline.java:179)
>     at 
> java.base/java.util.ArrayList$ArrayListSpliterator.forEachRemaining(ArrayList.java:1625)
>     at 
> java.base/java.util.stream.AbstractPipeline.copyInto(AbstractPipeline.java:509)
>     at 
> java.base/java.util.stream.AbstractPipeline.wrapAndCopyInto(AbstractPipeline.java:499)
>     at 
> java.base/java.util.stream.ForEachOps$ForEachOp.evaluateSequential(ForEachOps.java:150)
>     at 
> java.base/java.util.stream.ForEachOps$ForEachOp$OfRef.evaluateSequential(ForEachOps.java:173)
>     at 
> java.base/java.util.stream.AbstractPipeline.evaluate(AbstractPipeline.java:234)
>     at 
> java.base/java.util.stream.ReferencePipeline.forEach(ReferencePipeline.java:596)
>     at 
> io.debezium.relational.TableSchemaBuilder.create(TableSchemaBuilder.java:147) 
>    at 
> io.debezium.relational.RelationalDatabaseSchema.buildAndRegisterSchema(RelationalDatabaseSchema.java:135)
>     at 
> io.debezium.connector.mysql.MySqlDatabaseSchema.lambda$applySchemaChange$2(MySqlDatabaseSchema.java:179)
>     at java.base/java.lang.Iterable.forEach(Iterable.java:75)    at 
> io.debezium.connector.mysql.MySqlDatabaseSchema.applySchemaChange(MySqlDatabaseSchema.java:179)
>     at 
> org.apache.flink.cdc.connectors.mysql.debezium.dispatcher.EventDispatcherImpl$SchemaChangeEventReceiver.schemaChangeEvent(EventDispatcherImpl.java:314)
>     at 
> io.debezium.connector.mysql.MySqlStreamingChangeEventSource.lambda$handleQueryEvent$2(MySqlStreamingChangeEventSource.java:673)
>     ... 12 more
>  {code}
> !image-2025-06-27-12-31-26-011.png!
> As we can see, The mysql has cover it successfully ,but flink-cdc cannot 
> cover it `desc table `



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

Reply via email to