Hi Chris (and the wider Debezium community),
We're hitting a persistent Oracle DDL parser failure on a production XStream
connector and would appreciate a sanity check on root cause and accepted
workarounds before we go further.
# Environment
- Adapter: xstream (database.connection.adapter=xstream)
- Source: Oracle Database 19c, captured via XStream outbound server
- Kafka Connect 3.8 on Strimzi
- schema.history.internal.skip.unparseable.ddl=true (already)
# Symptom
Any LOB column whose segment is named (either user-supplied or
SYS_LOB...$$ system-generated) produces a CREATE TABLE statement from
DBMS_METADATA.GET_DDL that the Oracle DDL parser rejects. Reduced repro:
CREATE TABLE T (X CLOB)
LOB ("X") STORE AS BASICFILE "X_SEG"
(ENABLE STORAGE IN ROW CHUNK 8192 RETENTION NOCACHE LOGGING);
Real-world failing table (one of several):
CREATE TABLE "S"."OUTREACH_PAYLOAD"
( ...,
"OUTREACHPAYLOAD" CLOB,
...,
CONSTRAINT "OUTREACH_PAYLOAD_PK" PRIMARY KEY ("UIDPK") USING INDEX ENABLE,
SUPPLEMENTAL LOG DATA (ALL) COLUMNS,
SUPPLEMENTAL LOG DATA (PRIMARY KEY) COLUMNS,
SUPPLEMENTAL LOG DATA (UNIQUE INDEX) COLUMNS,
SUPPLEMENTAL LOG DATA (FOREIGN KEY) COLUMNS
)
LOB ("OUTREACHPAYLOAD") STORE AS BASICFILE "OUTREACH_JSON_PAYLOAD"
(ENABLE STORAGE IN ROW CHUNK 8192 RETENTION NOCACHE LOGGING);
ANTLR reports:
mismatched input '"OUTREACH_JSON_PAYLOAD"' expecting {<EOF>, '/', ';'}
Stack (trimmed):
io.debezium.text.ParsingException: DDL statement couldn't be parsed...
at io.debezium.antlr.ParsingErrorListener.syntaxError(...)
at io.debezium.connector.oracle.antlr.OracleDdlParser.parseTree(...)
at io.debezium.connector.oracle.antlr.OracleDdlParser.parse(OracleDdlParser.java:70)
at io.debezium.connector.oracle.OracleSchemaChangeEventEmitter.emitSchemaChangeEvent(OracleSchemaChangeEventEmitter.java:104)
at io.debezium.pipeline.EventDispatcher.dispatchSchemaChangeEvent(EventDispatcher.java:417)
at io.debezium.connector.oracle.xstream.LcrEventHandler.dispatchDataChangeEvent(LcrEventHandler.java:184)
at io.debezium.connector.oracle.xstream.LcrEventHandler.resolveAndDispatchCurrentChunkedRow(...)
at io.debezium.connector.oracle.xstream.LcrEventHandler.processChunk(...)
at oracle.streams.XStreamOut.XStreamOutReceiveLCRCallbackNative(Native Method)
# Why it isn't "just a parse warning"
The connector keeps running (skip.unparseable.ddl=true), but because the
parse never succeeds the schema cache for the table never gets populated.
Every chunked LOB LCR for the table re-enters LcrEventHandler ->
dispatchSchemaChangeEvent -> OracleSchemaChangeEventEmitter, which:
1. logs:
"Obtaining schema for table <schema>.<table>, which should be
already loaded, this may signal potential bug in fetching table
schemas."
2. calls DBMS_METADATA.GET_DDL on the source DB,
3. re-runs the parser, which fails again,
4. the skip flag swallows it, cache stays empty.
So we end up with:
- A measurable AWR signature on the source DB:
DBMS_METADATA.GET_DDL is a top CPU consumer during DML-heavy windows
on the affected table.
- Continuous WARN/ERROR storms in Connect logs proportional to chunked
LOB DML volume (not to
schema.refresh.interval.ms).
Bumping
schema.refresh.interval.ms from 10s -> 5m did not help, which
matches the stack: the trigger is per-LCR cache-miss, not the periodic
timer.
# What we believe the root cause is
PlSqlParser.g4's lob_storage / lob_segname rule accepts the segment-name
position only as an unquoted regular_id. DBMS_METADATA.GET_DDL always
emits the segment name as a DELIMITED_ID (quoted), regardless of whether
the user supplied it or Oracle generated it.
Sketch of the change that would fix it:
lob_segname
: regular_id
| DELIMITED_ID
;
(or equivalent: allow an optional id_expression between
'BASICFILE'/'SECUREFILE' and the parenthesised parameter list).
Adding three OracleDdlParserTest cases (BASICFILE quoted, SECUREFILE
quoted, SYS_LOB...$$) would cover the common shapes.
# Questions
1. Does the diagnosis above match what you'd expect, or is there a
code path on the Oracle XStream side that should be caching past
a failed parse so the loop doesn't fire forever?
2. Is there a config knob we missed that prevents per-LCR schema
reconciliation for a known-good column set?
3. Any interim workaround you'd recommend beyond
skip.unparseable.ddl=true?
Thanks for the great work on the Oracle connector.
Regards,
Shahul