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

DDL ParsingException - "SUPPLEMENTAL LOG DATA (UNIQUE INDEX) COLUMNS"

    XMLWordPrintable

Details

    Description

      We tested the connector (1.6.0.Beta2) and caught an error:

      [2021-06-11 10:10:44,712] INFO Table ABCDR.ABCD_SHARD_1_3.D_PLAN_SCHEDULE_LOT_ENTRY is new and will be captured. (io.debezium.connector.oracle.xstream.LcrEventHandler:160)
      [2021-06-11 10:10:44,761] INFO Database Version: Oracle Database 12c Enterprise Edition Release 12.2.0.1.0 - 64bit Production (io.debezium.connector.oracle.OracleConnection:71)
      [2021-06-11 10:10:45,095] INFO Connection gracefully closed (io.debezium.jdbc.JdbcConnection:951)
      [2021-06-11 10:10:45,096] INFO KafkaDatabaseHistory Consumer config: {enable.auto.commit=false, value.deserializer=org.apache.kafka.common.serialization.StringDeserializer, group.id=ABCDf2-dbhistory, auto.offset.reset=earliest, session.timeout.ms=10000, bootstrap.servers=10.204.192.111:9092,10.204.192.112:9092,10.204.192.113:9092, client.id=ABCDf2-dbhistory, key.deserializer=org.apache.kafka.common.serialization.StringDeserializer, fetch.min.bytes=1} (io.debezium.relational.history.KafkaDatabaseHistory:215)
      [2021-06-11 10:10:45,096] INFO KafkaDatabaseHistory Producer config: {bootstrap.servers=10.204.192.111:9092,10.204.192.112:9092,10.204.192.113:9092, value.serializer=org.apache.kafka.common.serialization.StringSerializer, buffer.memory=1048576, retries=1, key.serializer=org.apache.kafka.common.serialization.StringSerializer, client.id=ABCDf2-dbhistory, linger.ms=0, batch.size=32768, max.block.ms=10000, acks=1} (io.debezium.relational.history.KafkaDatabaseHistory:216)
      [2021-06-11 10:10:45,096] INFO Requested thread factory for connector OracleConnector, id = ABCDf2 named = db-history-config-check (io.debezium.util.Threads:270)
      [2021-06-11 10:10:45,098] ERROR Producer failure (io.debezium.pipeline.ErrorHandler:31)
      io.debezium.text.ParsingException: DDL statement couldn't be parsed. Please open a Jira issue with the statement '
        CREATE TABLE "ABCD_SHARD_1_3"."D_PLAN_SCHEDULE_LOT_ENTRY"
         (    "ID" NUMBER(38,0) NOT NULL ENABLE,
              "LOT_ID" NUMBER(38,0) NOT NULL ENABLE,
              "PLAN_SCHEDULE_ID" NUMBER(38,0) NOT NULL ENABLE,
              "IS_ACTUAL" NUMBER(1,0) NOT NULL ENABLE,
              "OOS_POSITION_NUMBER" NVARCHAR2(50) DEFAULT 0,
               CONSTRAINT "PK_PSLE_ENTRY_ID" PRIMARY KEY ("ID")
        USING INDEX  ENABLE,
               SUPPLEMENTAL LOG DATA (PRIMARY KEY) COLUMNS,
               SUPPLEMENTAL LOG DATA (UNIQUE INDEX) COLUMNS,
               SUPPLEMENTAL LOG DATA (FOREIGN KEY) COLUMNS,
               CONSTRAINT "FK_PSLE_LOT_VERSION" FOREIGN KEY ("LOT_ID")
                REFERENCES "ABCD_SHARD_1_3"."D_LOT_VERSION" ("ID") DISABLE,
               CONSTRAINT "FK_PSLE_PLAN_SCHEDULE_VERSION" FOREIGN KEY ("PLAN_SCHEDULE_ID")
                REFERENCES "ABCD_SHARD_1_3"."D_PLAN_SCHEDULE_VERSION" ("ID") ENABLE
         ) ;'
      extraneous input 'INDEX' expecting {'ALL', 'FOREIGN', 'PRIMARY', 'UNIQUE', ')', ','}
              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.reportUnwantedToken(DefaultErrorStrategy.java:377)
              at org.antlr.v4.runtime.DefaultErrorStrategy.sync(DefaultErrorStrategy.java:275)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.supplemental_id_key_clause(PlSqlParser.java:58187)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.supplemental_logging_props(PlSqlParser.java)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.relational_property(PlSqlParser.java:50993)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.relational_table(PlSqlParser.java:50728)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.create_table(PlSqlParser.java:49055)
              at io.debezium.ddl.parser.oracle.generated.PlSqlParser.unit_statement(PlSqlParser.java:2421)
              at io.debezium.connector.oracle.antlr.OracleDdlParser.parseTree(OracleDdlParser.java:68)
              at io.debezium.connector.oracle.antlr.OracleDdlParser.parseTree(OracleDdlParser.java:32)
              at io.debezium.antlr.AntlrDdlParser.parse(AntlrDdlParser.java:82)
              at io.debezium.connector.oracle.antlr.OracleDdlParser.parse(OracleDdlParser.java:63)
              at io.debezium.connector.oracle.OracleSchemaChangeEventEmitter.emitSchemaChangeEvent(OracleSchemaChangeEventEmitter.java:81)
              at io.debezium.pipeline.EventDispatcher.dispatchSchemaChangeEvent(EventDispatcher.java:297)
              at io.debezium.connector.oracle.xstream.LcrEventHandler.dispatchDataChangeEvent(LcrEventHandler.java:161)
              at io.debezium.connector.oracle.xstream.LcrEventHandler.processRowLCR(LcrEventHandler.java:143)
              at io.debezium.connector.oracle.xstream.LcrEventHandler.processLCR(LcrEventHandler.java:111)
              at oracle.streams.XStreamOut.XStreamOutReceiveLCRCallbackNative(Native Method)
              at oracle.streams.XStreamOut.receiveLCRCallback(XStreamOut.java:465)
              at io.debezium.connector.oracle.xstream.XstreamStreamingChangeEventSource.execute(XstreamStreamingChangeEventSource.java:105)
              at io.debezium.connector.oracle.xstream.XstreamStreamingChangeEventSource.execute(XstreamStreamingChangeEventSource.java:42)
              at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:159)
              at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:122)
              at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
              at java.util.concurrent.FutureTask.run(FutureTask.java:266)
              at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
              at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
              at java.lang.Thread.run(Thread.java:748)
      

      Connector config:

      { 
      "name": "XOUT_PLAN_SCHEDULE_110621_v0", 
      "config": { 
      "connector.class": "io.debezium.connector.oracle.OracleConnector", 
      "tasks.max": "1", 
      "snapshot.mode": "schema_only", 
      "snapshot.delay.ms": "5000", 
      "heartbeat.interval.ms": "1000", 
      "provide.transaction.metadata": "true", 
      "decimal.handling.mode": "string",
      "sanitize.field.names": "true",
      "event.processing.failure.handling.mode": "warn", 
      "database.server.name": "ABCDf2", 
      "database.hostname": "192.168.2.69", 
      "database.port": "1521", 
      "database.user": "xstrm", 
      "database.password": "xstrm", 
      "database.dbname": "ABCDf2", 
      "database.out.server.name": "XOUT_PLAN_SCHEDULE", 
      "database.history.kafka.bootstrap.servers": "10.204.192.111:9092,10.204.192.112:9092,10.204.192.113:9092", 
      "database.history.kafka.topic": "history_PLAN_SCHEDULE_11621_v0",  
      "database.connection.adapter": "xstream",
      "rac.nodes": "192.168.2.69,192.168.2.70",
      "table.include.list": "ABCD_SHARD_1_3.D_PLAN_SCHEDULE_LOT_ENTRY,ABCD_SHARD_1_3.D_PLAN_SCHEDULE_VERSION,ABCD_SHARD_1_3.D_PLAN_SCHEDULE_ENTITY" } } 
      

      Attachments

        Activity

          People

            ccranfor@redhat.com Chris Cranford
            chapeauclaque Dmitriy Rum (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: