Qishang Zhong created FLINK-34905:
-------------------------------------

             Summary: The default length of CHAR/BINARY data type of Add column 
DDL
                 Key: FLINK-34905
                 URL: https://issues.apache.org/jira/browse/FLINK-34905
             Project: Flink
          Issue Type: Bug
          Components: Flink CDC
            Reporter: Qishang Zhong


I run the DDL in mysql
{code:java}
ALTER TABLE test.products ADD Column1 BINARY NULL;  
ALTER TABLE test.products ADD Column2 CHAR NULL; {code}
Encountered the follow error:
{code:java}

Caused by: java.lang.IllegalArgumentException: Binary string length must be 
between 1 and 2147483647 (both inclusive).
        at 
org.apache.flink.cdc.common.types.BinaryType.<init>(BinaryType.java:53)
        at 
org.apache.flink.cdc.common.types.BinaryType.<init>(BinaryType.java:61)
        at org.apache.flink.cdc.common.types.DataTypes.BINARY(DataTypes.java:42)
        at 
org.apache.flink.cdc.connectors.mysql.utils.MySqlTypeUtils.convertFromColumn(MySqlTypeUtils.java:221)
        at 
org.apache.flink.cdc.connectors.mysql.utils.MySqlTypeUtils.fromDbzColumn(MySqlTypeUtils.java:111)
        at 
org.apache.flink.cdc.connectors.mysql.source.parser.CustomAlterTableParserListener.toCdcColumn(CustomAlterTableParserListener.java:256)
        at 
org.apache.flink.cdc.connectors.mysql.source.parser.CustomAlterTableParserListener.lambda$exitAlterByAddColumn$0(CustomAlterTableParserListener.java:126)
        at 
io.debezium.connector.mysql.antlr.MySqlAntlrDdlParser.runIfNotNull(MySqlAntlrDdlParser.java:358)
        at 
org.apache.flink.cdc.connectors.mysql.source.parser.CustomAlterTableParserListener.exitAlterByAddColumn(CustomAlterTableParserListener.java:98)
        at 
io.debezium.ddl.parser.mysql.generated.MySqlParser$AlterByAddColumnContext.exitRule(MySqlParser.java:15459)
        at 
io.debezium.antlr.ProxyParseTreeListenerUtil.delegateExitRule(ProxyParseTreeListenerUtil.java:64)
        at 
org.apache.flink.cdc.connectors.mysql.source.parser.CustomMySqlAntlrDdlParserListener.exitEveryRule(CustomMySqlAntlrDdlParserListener.java:124)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.exitRule(ParseTreeWalker.java:48)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:30)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:28)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:28)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:28)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:28)
        at 
org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:28)
        at io.debezium.antlr.AntlrDdlParser.parse(AntlrDdlParser.java:87)
        at 
org.apache.flink.cdc.connectors.mysql.source.MySqlEventDeserializer.deserializeSchemaChangeRecord(MySqlEventDeserializer.java:88)
        at 
org.apache.flink.cdc.debezium.event.SourceRecordEventDeserializer.deserialize(SourceRecordEventDeserializer.java:52)
        at 
org.apache.flink.cdc.debezium.event.DebeziumEventDeserializationSchema.deserialize(DebeziumEventDeserializationSchema.java:93)
        at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlRecordEmitter.emitElement(MySqlRecordEmitter.java:119)
        at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlRecordEmitter.processElement(MySqlRecordEmitter.java:96)
        at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlPipelineRecordEmitter.processElement(MySqlPipelineRecordEmitter.java:120)
        at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlRecordEmitter.emitRecord(MySqlRecordEmitter.java:73)
        at 
org.apache.flink.cdc.connectors.mysql.source.reader.MySqlRecordEmitter.emitRecord(MySqlRecordEmitter.java:46)
        at 
org.apache.flink.connector.base.source.reader.SourceReaderBase.pollNext(SourceReaderBase.java:160)
        at 
org.apache.flink.streaming.api.operators.SourceOperator.emitNext(SourceOperator.java:419)
        at 
org.apache.flink.streaming.runtime.io.StreamTaskSourceInput.emitNext(StreamTaskSourceInput.java:68)
        at 
org.apache.flink.streaming.runtime.io.StreamOneInputProcessor.processInput(StreamOneInputProcessor.java:65)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.processInput(StreamTask.java:562)
        at 
org.apache.flink.streaming.runtime.tasks.mailbox.MailboxProcessor.runMailboxLoop(MailboxProcessor.java:231)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.runMailboxLoop(StreamTask.java:858)
        at 
org.apache.flink.streaming.runtime.tasks.StreamTask.invoke(StreamTask.java:807)
        at 
org.apache.flink.runtime.taskmanager.Task.runWithSystemExitMonitoring(Task.java:953)
        at 
org.apache.flink.runtime.taskmanager.Task.restoreAndInvoke(Task.java:932)
        at org.apache.flink.runtime.taskmanager.Task.doRun(Task.java:746)
        at org.apache.flink.runtime.taskmanager.Task.run(Task.java:562)
        at java.lang.Thread.run(Thread.java:750) {code}
 



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

Reply via email to