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

DDL statement with TokuDB engine specific "CLUSTERING KEY" couldn't be parsed

    XMLWordPrintable

Details

    • False
    • None
    • False

    Description

      What Debezium connector do you use and what version?

      mysql-connector 1.9.4 Final

      What is the connector configuration?

      {
        "name": "connector1",
        "config": {
          "connector.class": "io.debezium.connector.mysql.MySqlConnector",
          "snapshot.locking.mode": "none",
          "database.user": "user",
          "database.server.id": "10",
          "tasks.max": "1",
          "database.server.name": "dbz-19_servername",
          "database.port": "3306",
          "include.schema.changes": "false",
          "table.include.list": "db.table",
          "key.converter.schemas.enable": "false",
          "database.hostname": "host",
          "database.password": "pass",
          "value.converter.schemas.enable": "false",
          "name": "connector1",
          "database.whitelist": "db",
          "snapshot.mode": "when_needed",
          "database.history.skip.unparseable.ddl": "true",
          "database.history.kafka.bootstrap.servers": "bootstrap.servers_string",
          "database.history.kafka.topic": "dbhistory.topic1",
          "database.history.producer.security.protocol": "security_protocol_string",
          "database.history.consumer.security.protocol": "security_protocol_string",
          "database.history.consumer.sasl.mechanism": "sasl_mechanism_string",
          "database.history.producer.sasl.mechanism": "sasl_mechanism_string",
          "database.history.producer.sasl.jaas.config": "sasl_jaas_config_string",
          "database.history.consumer.sasl.jaas.config": "sasl_jaas_config_string"
        }
      }
      

      What is the captured database version and mode of depoyment?

      (E.g. on-premises, with a specific cloud provider, etc.)

      MySQL Percona Edition with TokuDB engine
      5.7.38-41-log Percona Server (GPL), Release 41

      on-premises

      What behaviour do you expect?

      Connector read table schema with specific TokuDB engine keyword "CLUSTERING KEY" and start after deploy.

      CREATE TABLE `table1` (
        `col1` int(10) unsigned NOT NULL,
        `col2` varchar(32) NOT NULL,
        `col3` enum('var1','var2','var3') NOT NULL,
        PRIMARY KEY (`col1`,`col2`,`col3`),
        CLUSTERING KEY `clkey1` (`col3`,`col2`)
      ) ENGINE=TokuDB DEFAULT CHARSET=cp1251

      What behaviour do you see?

      Connector read table schema with specific TokuDB engine keyword "CLUSTERING KEY"  and failed to start with message "DDL statement couldn't be parsed"

      Do you see the same behaviour using the latest relesead Debezium version?

      (Ideally, also verify with latest Alpha/Beta/CR version)

      No

      Do you have the connector logs, ideally from start till finish?

      (You might be asked later to provide DEBUG/TRACE level log)

      io.debezium.DebeziumException: io.debezium.text.ParsingException: DDL statement couldn't be parsed. Please open a Jira issue with the statement 'CREATE TABLE `table1` (
        `col1` int(10) unsigned NOT NULL,
        `col2` varchar(32) NOT NULL,
        `col3` enum('var1','var2','var3') NOT NULL,
        PRIMARY KEY (`col1`,`col2`,`col3`),
        CLUSTERING KEY `clkey1` (`col3`,`col2`)
      ) ENGINE=TokuDB DEFAULT CHARSET=cp1251'
      no viable alternative at input 'CREATE TABLE `table1` (\n  `col1` int(10) unsigned NOT NULL,\n  `col2` varchar(32) NOT NULL,\n  `col3` enum('var1','var2','var3') NOT NULL,\n  PRIMARY KEY (`col1`,`col2`,`col3`),\n  CLUSTERING KEY'
          at io.debezium.pipeline.source.AbstractSnapshotChangeEventSource.execute(AbstractSnapshotChangeEventSource.java:85)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.doSnapshot(ChangeEventSourceCoordinator.java:155)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:137)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:109)
          at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
          at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
          at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
          at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
          at java.base/java.lang.Thread.run(Thread.java:829)
      Caused by: io.debezium.text.ParsingException: DDL statement couldn't be parsed. Please open a Jira issue with the statement 'CREATE TABLE `table1` (
        `col1` int(10) unsigned NOT NULL,
        `col2` varchar(32) NOT NULL,
        `col3` enum('var1','var2','var3') NOT NULL,
        PRIMARY KEY (`col1`,`col2`,`col3`),
        CLUSTERING KEY `clkey1` (`col3`,`col2`)
      ) ENGINE=TokuDB DEFAULT CHARSET=cp1251'
      no viable alternative at input 'CREATE TABLE `table1` (\n  `col1` int(10) unsigned NOT NULL,\n  `col2` varchar(32) NOT NULL,\n  `col3` enum('var1','var2','var3') NOT NULL,\n  PRIMARY KEY (`col1`,`col2`,`col3`),\n  CLUSTERING KEY'
          at io.debezium.antlr.ParsingErrorListener.syntaxError(ParsingErrorListener.java:43)
          at org.antlr.v4.runtime.ProxyErrorListener.syntaxError(ProxyErrorListener.java:41)
          at org.antlr.v4.runtime.Parser.notifyErrorListeners(Parser.java:544)
          at org.antlr.v4.runtime.DefaultErrorStrategy.reportNoViableAlternative(DefaultErrorStrategy.java:310)
          at org.antlr.v4.runtime.DefaultErrorStrategy.reportError(DefaultErrorStrategy.java:136)
      

       

       

      Attachments

        Activity

          People

            anmohant Anisha Mohanty
            padner Kos Fp (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: