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

Reselect Post Processor not working when pkey of type uuid etc.

XMLWordPrintable

      In order to make your issue reports as actionable as possible, please provide the following information, depending on the issue type.

      Bug report

      For bug reports, provide this information, please:

      What Debezium connector do you use and what version?

      2.7.3.Finale

      What is the connector configuration?

      {
      	"connector.class": "io.debezium.connector.postgresql.PostgresConnector",
      	"max.queue.size": "32768",
      	"topic.creation.default.partitions": "3",
      	"timestampConverter.format.date": "yyyy-MM-dd",
      	"slot.name": "xx",
      	"publication.name": "xx",
      	"value.subject.name.strategy": "io.confluent.kafka.serializers.subject.TopicRecordNameStrategy",
      	"tombstones.on.delete": "true",
      	"topic.prefix": "pp",
      	"decimal.handling.mode": "double",
      	"timestampConverter.format.datetime": "yyyy-MM-dd'T'HH:mm:ss.SSS'Z'",
      	"topic.creation.default.replication.factor": "3",
      	"value.converter": "io.confluent.connect.avro.AvroConverter",
      	"key.converter": "io.confluent.connect.avro.AvroConverter",
      	"database.user": "xx",
      	"database.dbname": "xx",
      	"producer.override.compression.type": "snappy",
      	
      	"plugin.name": "pgoutput",
      	"topic.creation.enable": "true",
      	"value.converter.schema.registry.url": "xx",
      	"database.hostname": "***",
      	"database.password": "***",
      	"max.batch.size": "8192",
      	"table.include.list": "public.x",
      	"key.converter.schema.registry.url": "xxx",
      	"snapshot.mode": "never",
      	"slot.drop.on.stop": "true",
      	"post.processors": "reselector", 
        "reselector.type": "io.debezium.processors.reselect.ReselectColumnsPostProcessor", 
        "reselector.reselect.columns.include.list": "public.x:y", 
        "reselector.reselect.unavailable.values": "true", 
        "reselector.reselect.null.values": "true" ,
        "reselector.reselect.use.event.key": "false" 
      }
      

      What is the captured database version and mode of deployment?

      AWS RDS

      What behavior do you expect?

      reselect should work when `y` is of type uuid as well

      What behavior do you see?

      reselect post processor not worked and able to see `__debezium_unavailable`

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

      yes

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

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

      	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      	at java.base/java.lang.Thread.run(Thread.java:840)
      [2024-09-26 13:00:56,186] WARN [test-toast|task-0] Failed to re-select row for table public.shipment and key Struct{y=fd74cd88-24dd-41d9-8922-5150029025a3} (io.debezium.processors.reselect.ReselectColumnsPostProcessor:172)
      org.postgresql.util.PSQLException: ERROR: operator does not exist: uuid = character varying
        Hint: No operator matches the given name and argument types. You might need to add explicit type casts.
        Position: 76
      	at org.postgresql.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2725)
      	at org.postgresql.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:2412)
      	at org.postgresql.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:371)
      	at org.postgresql.jdbc.PgStatement.executeInternal(PgStatement.java:502)
      	at org.postgresql.jdbc.PgStatement.execute(PgStatement.java:419)
      	at org.postgresql.jdbc.PgPreparedStatement.executeWithFlags(PgPreparedStatement.java:194)
      	at org.postgresql.jdbc.PgPreparedStatement.executeQuery(PgPreparedStatement.java:137)
      	at io.debezium.jdbc.JdbcConnection.prepareQuery(JdbcConnection.java:808)
      	at io.debezium.jdbc.JdbcConnection.reselectColumns(JdbcConnection.java:1673)
      	at io.debezium.jdbc.JdbcConnection.reselectColumns(JdbcConnection.java:1668)
      	at io.debezium.processors.reselect.ReselectColumnsPostProcessor.apply(ReselectColumnsPostProcessor.java:165)
      	at io.debezium.pipeline.EventDispatcher.doPostProcessing(EventDispatcher.java:745)
      	at io.debezium.pipeline.EventDispatcher$StreamingChangeRecordReceiver.changeRecord(EventDispatcher.java:506)
      	at io.debezium.pipeline.EventDispatcher$2.changeRecord(EventDispatcher.java:304)
      	at io.debezium.relational.RelationalChangeRecordEmitter.emitUpdateRecord(RelationalChangeRecordEmitter.java:124)
      	at io.debezium.relational.RelationalChangeRecordEmitter.emitChangeRecords(RelationalChangeRecordEmitter.java:53)
      	at io.debezium.connector.postgresql.PostgresChangeRecordEmitter.emitChangeRecords(PostgresChangeRecordEmitter.java:94)
      	at io.debezium.pipeline.EventDispatcher.dispatchDataChangeEvent(EventDispatcher.java:275)
      	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processReplicationMessages(PostgresStreamingChangeEventSource.java:318)
      	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.lambda$processMessages$0(PostgresStreamingChangeEventSource.java:217)
      	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.decodeUpdate(PgOutputMessageDecoder.java:479)
      	at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.processNotEmptyMessage(PgOutputMessageDecoder.java:211)
      	at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.processMessage(AbstractMessageDecoder.java:41)
      	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:642)
      	at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:634)
      	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:217)
      	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:179)
      	at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:42)
      	at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:313)
      	at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:203)
      	at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:143)
      	at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:539)
      	at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
      	at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1136)
      	at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:635)
      	at java.base/java.lang.Thread.run(Thread.java:840)
      

      How to reproduce the issue using our tutorial deployment?

      have a table with primary key uuid datatype and a toast column

      Feature request or enhancement

      For feature requests or enhancements, provide this information, please:

      Which use case/requirement will be addressed by the proposed feature?

      <Your answer>

      Implementation ideas (optional)

      as per zulip discussion, we will now pass `Table` in reselect column so casting can be done in PreparedStatement builder directly

            Unassigned Unassigned
            gaurav7261 Gaurav Miglani
            Votes:
            0 Vote for this issue
            Watchers:
            3 Start watching this issue

              Created:
              Updated:
              Resolved: