oracle clob update (__debezium_unavailable_value)

10 views
Skip to first unread message

Alex Epifanov

unread,
Jul 28, 2025, 11:53:48 AMJul 28
to debe...@googlegroups.com

Hello debezium team. I need to migrate data from oracle to postgres. I use debezium-connector-oracle-3.2.0.Final for source oracle connector.
I created a table on my source side (oracle).

---------
create table alex.t7(id7 number primary key, name7 varchar2(100), message7 CLOB);
alter table alex.t7 add supplemental log data (all) columns;
insert into alex.t7(id7,name7,message7) values(1,'one','asdasASSADA');
insert into alex.t7(id7,name7,message7) values(2,'two','SDFSDFSDF');
insert into alex.t7(id7,name7,message7) values(3,'three','SGERWS');
commit;

update alex.t7 set name7='YYYY' where id7=3;
commit;
-----------

But on my target side (postgres) after update I see value "__debezium_unavailable_value" in column message7 for id7=3.
I decided to use "post.processors" : "reselector" in my source config

You can see part of my source config below

 "table.include.list" : "alex.t7",
 "lob.enabled" : "true",
 "post.processors" : "reselector",
 "reselector.type" : "io.debezium.processors.reselect.ReselectColumnsPostProcessor",
 "reselector.reselect.columns.include.list" : "ALEX.T7:MESSAGE7",

But in my logs I see errors "Invalid column type" 

[2025-07-28 16:53:37,078] DEBUG [oradbc7|task-0] Adding column MESSAGE7 for table MIGRPG.ALEX.T7 to re-select list due to unavailable value placeholder. (io.debezium.processors.reselect.ReselectColumnsPostProcessor:282)
[2025-07-28 16:53:37,079] TRACE [oradbc7|task-0] Inserting prepared statement 'SELECT "MESSAGE7" FROM (SELECT * FROM "ALEX"."T7" AS OF SCN ?) WHERE ID7=?' removed from the cache (io.debezium.jdbc.JdbcConnection:1494)
[2025-07-28 16:53:37,079] TRACE [oradbc7|task-0] PreparedStatement 'SELECT "MESSAGE7" FROM (SELECT * FROM "ALEX"."T7" AS OF SCN ?) WHERE ID7=?' with 600s timeout (io.debezium.jdbc.JdbcConnection:1498)
[2025-07-28 16:53:37,080] WARN [oradbc7|task-0] Failed to re-select columns for table MIGRPG.ALEX.T7 and key [Struct{scale=0,value=[B@158ad6ab}] (io.debezium.processors.reselect.ReselectColumnsPostProcessor:236)
java.sql.SQLException: Invalid column type
        at oracle.jdbc.driver.OraclePreparedStatement.setObjectCritical(OraclePreparedStatement.java:8804)
        at oracle.jdbc.driver.OraclePreparedStatement.setObjectInternal(OraclePreparedStatement.java:8286)
        at oracle.jdbc.driver.OraclePreparedStatement.setObjectInternal(OraclePreparedStatement.java:9061)
        at oracle.jdbc.driver.OraclePreparedStatement.setObject(OraclePreparedStatement.java:9036)
        at oracle.jdbc.driver.OraclePreparedStatementWrapper.setObject(OraclePreparedStatementWrapper.java:222)
        at io.debezium.jdbc.JdbcConnection.prepareQuery(JdbcConnection.java:811)
        at io.debezium.jdbc.JdbcConnection.reselectColumns(JdbcConnection.java:1724)
        at io.debezium.connector.oracle.OracleConnection.reselectColumns(OracleConnection.java:665)
        at io.debezium.processors.reselect.ReselectColumnsPostProcessor.apply(ReselectColumnsPostProcessor.java:223)
        at io.debezium.pipeline.EventDispatcher.doPostProcessing(EventDispatcher.java:785)
        at io.debezium.pipeline.EventDispatcher$StreamingChangeRecordReceiver.changeRecord(EventDispatcher.java:530)
        at io.debezium.pipeline.EventDispatcher$2.changeRecord(EventDispatcher.java:320)
        at io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:124)
        at io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:53)
        at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:290)
        at io.debezium.connector.oracle.logminer.buffered.BufferedLogMinerStreamingChangeEventSource.lambda$handleCommitEvent$0(BufferedLogMinerStreamingChangeEventSource.java:456)
        at io.debezium.connector.oracle.logminer.TransactionCommitConsumer.dispatchChangeEvent(TransactionCommitConsumer.java:506)
        at io.debezium.connector.oracle.logminer.TransactionCommitConsumer.prepareAndDispatch(TransactionCommitConsumer.java:329)
        at io.debezium.connector.oracle.logminer.TransactionCommitConsumer.close(TransactionCommitConsumer.java:109)
        at io.debezium.connector.oracle.logminer.buffered.BufferedLogMinerStreamingChangeEventSource.handleCommitEvent(BufferedLogMinerStreamingChangeEventSource.java:471)
        at io.debezium.connector.oracle.logminer.AbstractLogMinerStreamingChangeEventSource.processEvent(AbstractLogMinerStreamingChangeEventSource.java:471)
        at io.debezium.connector.oracle.logminer.AbstractLogMinerStreamingChangeEventSource.executeAndProcessQuery(AbstractLogMinerStreamingChangeEventSource.java:397)
        at io.debezium.connector.oracle.logminer.buffered.BufferedLogMinerStreamingChangeEventSource.process(BufferedLogMinerStreamingChangeEventSource.java:231)
        at io.debezium.connector.oracle.logminer.buffered.BufferedLogMinerStreamingChangeEventSource.executeLogMiningStreaming(BufferedLogMinerStreamingChangeEventSource.java:157)
        at io.debezium.connector.oracle.logminer.AbstractLogMinerStreamingChangeEventSource.execute(AbstractLogMinerStreamingChangeEventSource.java:209)
        at io.debezium.connector.oracle.logminer.AbstractLogMinerStreamingChangeEventSource.execute(AbstractLogMinerStreamingChangeEventSource.java:88)
        at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:326)
        at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:207)
        at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:147)
        at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:545)
        at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:328)
        at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1095)
        at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:619)
        at java.base/java.lang.Thread.run(Thread.java:1447)

Please, help me to solve this problem.


Chris Cranford

unread,
Jul 28, 2025, 12:39:28 PMJul 28
to debe...@googlegroups.com
Hi Alex -

That's for the report. What you're experiencing is a bug in the ReselectPostProcessor when working with VariableScaleDecimal types. I've logged DBZ-9293 [1] to address that.

As a workaround, if possible, you can use NUMBER with a length less-than 19 so that the value is encoded in an INT64, INT32, INT16, or INT8 data type. Those should work.

Thanks,
-cc

[1]: https://issues.redhat.com/browse/DBZ-9293
--
You received this message because you are subscribed to the Google Groups "debezium" group.
To unsubscribe from this group and stop receiving emails from it, send an email to debezium+u...@googlegroups.com.
To view this discussion visit https://groups.google.com/d/msgid/debezium/CAPxzvE4Cwj%3D%3DDUkMuFpk0c%2BTkmsKWJKRDL%3D4KK0aA2KRMHPgeg%40mail.gmail.com.

Alex Epifanov

unread,
Jul 28, 2025, 3:46:56 PMJul 28
to debe...@googlegroups.com
Thanks.

I verified with number(16) and post.processor worked well.

пн, 28 июл. 2025 г. в 19:39, Chris Cranford <cran...@gmail.com>:
Reply all
Reply to author
Forward
0 new messages