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

Primary key changes cause UnsupportedOperationException

      We are using Debezium 0.8.3.Final and we are running into an UnsupportedOperationException.

      Our latest database migration changes the primary key of a table.

      Before
      COLUMN_A, COLUMN_B, COLUMN_C
      Primary key: (COLUMN_A, COLUMN_C)

      After
      COLUMN_A, COLUMN_B, COLUMN_D
      Primary key: (COLUMN_A, COLUMN_D)

      Exception

      2018-11-19 19:43:39,455 ERROR || WorkerSourceTask{id=amfori-connector-0} Task threw an uncaught and unrecoverable exception [org.apache.kafka.connect.runtime.WorkerTask]
      org.apache.kafka.connect.errors.ConnectException: An exception ocurred in the change event producer. This connector will be stopped.
      at io.debezium.connector.base.ChangeEventQueue.throwProducerFailureIfPresent(ChangeEventQueue.java:168)
      at io.debezium.connector.base.ChangeEventQueue.poll(ChangeEventQueue.java:149)
      at io.debezium.connector.postgresql.PostgresConnectorTask.poll(PostgresConnectorTask.java:146)
      at org.apache.kafka.connect.runtime.WorkerSourceTask.execute(WorkerSourceTask.java:186)
      at org.apache.kafka.connect.runtime.WorkerTask.doRun(WorkerTask.java:170)
      at org.apache.kafka.connect.runtime.WorkerTask.run(WorkerTask.java:214)
      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)
      Caused by: java.lang.UnsupportedOperationException
      at java.util.Collections$UnmodifiableCollection$1.remove(Collections.java:1044)
      at io.debezium.relational.TableEditorImpl.updatePrimaryKeys(TableEditorImpl.java:103)
      at io.debezium.relational.TableEditorImpl.setColumns(TableEditorImpl.java:94)
      at io.debezium.connector.postgresql.RecordsStreamProducer.tableFromFromMessage(RecordsStreamProducer.java:528)
      at io.debezium.connector.postgresql.RecordsStreamProducer.columnValues(RecordsStreamProducer.java:425)
      at io.debezium.connector.postgresql.RecordsStreamProducer.process(RecordsStreamProducer.java:246)
      at io.debezium.connector.postgresql.RecordsStreamProducer.lambda$streamChanges$2(RecordsStreamProducer.java:128)
      at io.debezium.connector.postgresql.connection.wal2json.NonStreamingWal2JsonMessageDecoder.processMessage(NonStreamingWal2JsonMessageDecoder.java:62)
      at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.deserializeMessages(PostgresReplicationConnection.java:218)
      at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.read(PostgresReplicationConnection.java:203)
      at io.debezium.connector.postgresql.RecordsStreamProducer.streamChanges(RecordsStreamProducer.java:128)
      at io.debezium.connector.postgresql.RecordsStreamProducer.lambda$start$1(RecordsStreamProducer.java:114)
      ... 5 more
      

      Cause
      The TableEditorImpl::updatePrimaryKeys method seems to be calling List::remove on the unmodifiable list provided by TableEditorImpl::primaryKeyColumnNames

            [DBZ-997] Primary key changes cause UnsupportedOperationException

            Released

            Jiri Pechanec added a comment - Released

            That being said, it seems propagating an UnsupportedOperationException is something we should be able to improve upon.

            Gunnar Morling added a comment - That being said, it seems propagating an UnsupportedOperationException is something we should be able to improve upon.

            Indeed I'm afraid there's not much we can do here unfortunately. When detecting a schema change, the connector does an "out-of-bands" request via JDBC to obtain the current PK structure. This is needed as the messages from the DB themselves don't contain the information which columns form the PK. The connector docs discuss this caveat:

            The PostgreSQL connector retrieves the schema information as part of the events sent by the logical decoder plug-in. The only exception is the information about which columns compose the primary key, as this information is obtained from the JDBC metadata (side channel). If the primary key definition of a table changes (by adding, removing or renaming PK columns), then there exists a slight risk of an unfortunate timing when the primary key information from JDBC will not be synchronized with the change data in the logical decoding event and a small amount of messages will be created with an inconsistent key structure. If this happens then a restart of the connector and a reprocessing of the messages will fix the issue. To prevent the issue completely it is recommended to synchronize updates to the primary key structure with Debezium roughly using following sequence of operations:

            • Put the database or an application into a read-only mode
            • Let Debezium process all remaining events
            • Stop Debezium
            • Update the primary key definition
            • Put the database or the application into read/write state and start Debezium again

            This isn't very satisfactory but with the information we get from the database I can't see a better way at this point.

            Gunnar Morling added a comment - Indeed I'm afraid there's not much we can do here unfortunately. When detecting a schema change, the connector does an "out-of-bands" request via JDBC to obtain the current PK structure. This is needed as the messages from the DB themselves don't contain the information which columns form the PK. The connector docs discuss this caveat: The PostgreSQL connector retrieves the schema information as part of the events sent by the logical decoder plug-in. The only exception is the information about which columns compose the primary key, as this information is obtained from the JDBC metadata (side channel). If the primary key definition of a table changes (by adding, removing or renaming PK columns), then there exists a slight risk of an unfortunate timing when the primary key information from JDBC will not be synchronized with the change data in the logical decoding event and a small amount of messages will be created with an inconsistent key structure. If this happens then a restart of the connector and a reprocessing of the messages will fix the issue. To prevent the issue completely it is recommended to synchronize updates to the primary key structure with Debezium roughly using following sequence of operations: Put the database or an application into a read-only mode Let Debezium process all remaining events Stop Debezium Update the primary key definition Put the database or the application into read/write state and start Debezium again This isn't very satisfactory but with the information we get from the database I can't see a better way at this point.

            Hi, this is not a bug per se but by-product of https://issues.jboss.org/browse/DBZ-487
            We are getting PK information via JDBC call and it is possible with unfortunate timing that the JDBC information about primeray keys does not reflect the instatnt structure in change coming from wal decoder.

            Jiri Pechanec added a comment - Hi, this is not a bug per se but by-product of https://issues.jboss.org/browse/DBZ-487 We are getting PK information via JDBC call and it is possible with unfortunate timing that the JDBC information about primeray keys does not reflect the instatnt structure in change coming from wal decoder.

              jpechane Jiri Pechanec
              wouter.bancken.aca Wouter Bancken (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

                Created:
                Updated:
                Resolved: