Uploaded image for project: 'Debezium'
  1. Debezium
  2. DBZ-1204

Mysql: Getting ERROR `Failed due to error: connect.errors.ConnectException: For input string: "false"`

    Details

    • Type: Bug
    • Status: Closed (View Workflow)
    • Priority: Major
    • Resolution: Done
    • Affects Version/s: 0.9.0.Final, 0.9.1.Final, 0.9.2.Final, 0.9.3.Final
    • Fix Version/s: 0.9.4.Final
    • Component/s: mysql-connector
    • Labels:
      None

      Description

      I was running a mysql connector in schema only snapshot mode and It was up and running for 3 days and all of a sudden the connector stops working after throwing the below error:

      [2019-03-26 12:49:30,327] INFO Connected to MySQL binlog at <mysql server>:<mysql port>, starting at binlog file 'mysql-bin-changelog.084477', pos=85325481, skipping 0 events plus 1 rows (io.debezium.connector.mysql.BinlogReader:970)
      [2019-03-26 12:49:30,328] INFO Creating thread debezium-mysqlconnector-tran-binlog-client (io.debezium.util.Threads:263)
      [2019-03-26 12:49:30,380] ERROR Error during binlog processing. Last offset stored = null, binlog reader near position = mysql-bin-changelog.084477/85438787 (io.debezium.connector.mysql.BinlogReader:1020)
      [2019-03-26 12:49:30,380] ERROR Failed due to error: Error processing binlog event (io.debezium.connector.mysql.BinlogReader:209)
      org.apache.kafka.connect.errors.ConnectException: For input string: "false"
      	at io.debezium.connector.mysql.AbstractReader.wrap(AbstractReader.java:230)
      	at io.debezium.connector.mysql.AbstractReader.failed(AbstractReader.java:208)
      	at io.debezium.connector.mysql.BinlogReader.handleEvent(BinlogReader.java:477)
      	at com.github.shyiko.mysql.binlog.BinaryLogClient.notifyEventListeners(BinaryLogClient.java:1055)
      	at com.github.shyiko.mysql.binlog.BinaryLogClient.listenForEventPackets(BinaryLogClient.java:913)
      	at com.github.shyiko.mysql.binlog.BinaryLogClient.connect(BinaryLogClient.java:559)
      	at com.github.shyiko.mysql.binlog.BinaryLogClient$7.run(BinaryLogClient.java:793)
      	at java.lang.Thread.run(Thread.java:748)
      Caused by: java.lang.NumberFormatException: For input string: "false"
      	at java.lang.NumberFormatException.forInputString(NumberFormatException.java:65)
      	at java.lang.Integer.parseInt(Integer.java:580)
      	at java.lang.Short.parseShort(Short.java:118)
      	at java.lang.Short.parseShort(Short.java:144)
      	at io.debezium.jdbc.JdbcValueConverters.lambda$convertSmallInt$42(JdbcValueConverters.java:731)
      	at io.debezium.jdbc.JdbcValueConverters.convertValue(JdbcValueConverters.java:1165)
      	at io.debezium.jdbc.JdbcValueConverters.convertSmallInt(JdbcValueConverters.java:719)
      	at io.debezium.jdbc.JdbcValueConverters.lambda$converter$4(JdbcValueConverters.java:253)
      	at io.debezium.connector.mysql.antlr.listener.ColumnDefinitionParserListener.convertDefaultValueToSchemaType(ColumnDefinitionParserListener.java:263)
      	at io.debezium.connector.mysql.antlr.listener.ColumnDefinitionParserListener.enterDefaultValue(ColumnDefinitionParserListener.java:129)
      	at io.debezium.ddl.parser.mysql.generated.MySqlParser$DefaultValueContext.enterRule(MySqlParser.java:46498)
      	at io.debezium.antlr.ProxyParseTreeListenerUtil.delegateEnterRule(ProxyParseTreeListenerUtil.java:46)
      	at io.debezium.connector.mysql.antlr.listener.MySqlAntlrDdlParserListener.enterEveryRule(MySqlAntlrDdlParserListener.java:89)
      	at org.antlr.v4.runtime.tree.ParseTreeWalker.enterRule(ParseTreeWalker.java:41)
      	at org.antlr.v4.runtime.tree.ParseTreeWalker.walk(ParseTreeWalker.java:25)
      	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 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:85)
      	at io.debezium.connector.mysql.MySqlSchema.applyDdl(MySqlSchema.java:304)
      	at io.debezium.connector.mysql.BinlogReader.handleQueryEvent(BinlogReader.java:663)
      	at io.debezium.connector.mysql.BinlogReader.handleEvent(BinlogReader.java:461)
      	... 5 more
      

      This is the configuration of my task:

      curl -i -X POST -H "Accept:application/json" -H  "Content-Type:application/json" http://localhost:8084/connectors/ \
          -d '{
            "name": "mysql-connector-de",
            "config": {
                  "connector.class": "io.debezium.connector.mysql.MySqlConnector",
                  "database.hostname": "",
                  "database.history.kafka.recovery.attempts": 20,
                  "database.history.kafka.recovery.poll.interval.ms" : 10000000,
                  "database.port": "",
                  "database.user": "",
                  "database.password": "",
                  "database.server.id": "",
                  "database.server.name": "tran",
                  "database.whitelist": "db", "table.whitelist":"db.dummy",
                  "database.history.kafka.bootstrap.servers": "localhost:9092",
                  "database.history.kafka.topic": "dbhistory.tran",
                  "decimal.handling.mode":"string",
                  "snapshot.mode": "schema_only",
                  "rows.fetch.size": 2,
                  "include.schema.changes": "true"
             }
          }'
      

      The query which was causing the issue:

      CREATE TABLE IF NOT EXISTS `dbz_issue` (`id` INTEGER(11) NOT NULL auto_increment , `col1` INTEGER(11) NOT NULL UNIQUE, `col2` INTEGER(4) NOT NULL DEFAULT 1, `col3` INTEGER(11) NOT NULL, `col4` TINYINT(1) NOT NULL DEFAULT false, `col5` TINYINT(1) NOT NULL DEFAULT false, `created_on` DATETIME NOT NULL DEFAULT NOW(), `updated_on` DATETIME NOT NULL DEFAULT NOW(), `created_by_id` INTEGER(11) NOT NULL, `updated_by_id` INTEGER(11) NOT NULL, PRIMARY KEY (`id`));
      

      But this issue is only present in the versions > 0.9.0, I've tried the same with version 0.8.3 it's working fine without throwing any errors. Will you be able to fix the same in the upcoming release?

        Gliffy Diagrams

          Attachments

            Issue Links

              Activity

                People

                • Assignee:
                  jpechanec Jiri Pechanec
                  Reporter:
                  midhuns Midhun Sukumaran
                • Votes:
                  0 Vote for this issue
                  Watchers:
                  2 Start watching this issue

                  Dates

                  • Created:
                    Updated:
                    Resolved: