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

Ad-hoc blocking snapshot dies with "invalid snapshot identifier" immediately after connector creation

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?

      Debezium PostgreSQL v2.5.0.Final

      What is the connector configuration?

      # General connector setup
      name=SnapshotModeInitialCrash
      connector.class=io.debezium.connector.postgresql.PostgresConnector
      tasks.max=1
      
      # Basic database connection information
      database.hostname=10.10.100.1
      database.port=5432
      database.user=cdc_replication
      database.dbname=featureflags
      
      # Logical replication configuration
      publication.name=cdc_publication
      publication.autocreate.mode=disabled
      slot.name=snapshot_mode_initial_crash
      plugin.name=pgoutput
      
      # Disable snapshots on initialization to work around the bug of:
      # https://issues.redhat.com/browse/DBZ-7311
      # This is done by setting a non-existing table in the snapshot.include.collection.list
      snapshot.mode=initial
      snapshot.include.collection.list=DO_NOT_MATCH_ANY_TABLES_THIS_DOES_NOT_EXIST
      table.include.list=public.ffl_feature_flags
      topic.prefix=snapshot-mode-initial-crash
      
      # Signalling and notifications
      signal.enabled.channels=source
      signal.data.collection=framework.dbz_main_cdc_signal_v001
      
      key.converter=io.confluent.connect.avro.AvroConverter
      key.converter.schema.registry.url=http://10.10.100.1:17731
      value.converter=io.confluent.connect.avro.AvroConverter
      value.converter.schema.registry.url=http://10.10.100.1:17731 

      What is the captured database version and mode of depoyment?

      PG 13 running in a local docker container for local-only development.

      Steps to reproduce

      1. Create test database in PostgreSQL with a signal collection table and a sample data table to snapshot.
      2. Create a connector with the configuration similar to what I shared above.  The key part is to set snapshot.mode to initial, and set snapshot.include.collection.list to a non-sensical string that does not refer to any real table.  (this is being done to work around the bug of https://issues.redhat.com/browse/DBZ-7311). 
      3. Do not restarting the connector, or performing any other actions.
      4. Shortly after creating the connector, run SQL similar to the following to run an ad-hoc blocking snapshot.
      INSERT INTO framework.dbz_main_cdc_signal_v001 (signal_id, type, data)
      VALUES (gen_random_uuid()::text, 'execute-snapshot', '{"data-collections": ["public.ffl_feature_flags"], "type": "blocking"}')
      RETURNING signal_id;

      What behaviour do you expect?

      An ad-hoc blocking snapshot should be taken of the requested table as normal.

      What behaviour do you see?

      The connector logs the following error, and subsequently fails to gracefully restart.

      [2024-01-04 07:30:52,627] INFO [SnapshotModeInitialCrash|task-0] Opening transaction with statement SET TRANSACTION ISOLATION LEVEL REPEATABLE READ; SET TRANSACTION SNAPSHOT '0000001A-000000BB-1'; (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:249) [2024-01-04 07:30:52,660] ERROR [SnapshotModeInitialCrash|task-0] Error during snapshot (io.debezium.relational.RelationalSnapshotChangeEventSource:179) org.postgresql.util.PSQLException: ERROR: invalid snapshot identifier: "0000001A-000000BB-1" 

      It looks as if Debezium is trying to set the transaction snapshot to one that was not previously exported.  I am not sure how it came up with this snapshot identifier.

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

      (Ideally, also verify with latest Alpha/Beta/CR version)

      <Your answer>

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

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

      When the connector first starts after being created, this is what is logged.  It sounds relatively normal to me, but I include it in full in case there are any useful insights here.

       [2024-01-04 07:24:47,987] INFO [SnapshotModeInitialCrash|task-0] No previous offsets found (io.debezium.connector.common.BaseSourceTask:381)
      [2024-01-04 07:24:47,995] INFO [SnapshotModeInitialCrash|task-0] user 'cdc_replication' connected to database 'featureflags' on PostgreSQL 13.13 (Debian 13.13-1.pgdg120+1) on x86_64-pc-linux-gnu, compiled by gcc (Debian 12.2.0-14) 12.2.0, 64-bit with roles:
          role 'cdc_replication' [superuser: false, replication: true, inherit: true, create role: false, create db: false, can log in: true] (io.debezium.connector.postgresql.PostgresConnectorTask:120)
      [2024-01-04 07:24:47,999] INFO [SnapshotModeInitialCrash|task-0] Obtained valid replication slot ReplicationSlot [active=false, latestFlushedLsn=null, catalogXmin=null] (io.debezium.connector.postgresql.connection.PostgresConnection:325)
      [2024-01-04 07:24:47,999] INFO [SnapshotModeInitialCrash|task-0] No previous offset found (io.debezium.connector.postgresql.PostgresConnectorTask:129)
      [2024-01-04 07:24:48,043] INFO [SnapshotModeInitialCrash|task-0] Initializing PgOutput logical decoder publication (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:150)
      [2024-01-04 07:24:48,076] INFO [SnapshotModeInitialCrash|task-0] Creating replication slot with command CREATE_REPLICATION_SLOT "snapshot_mode_initial_crash"  LOGICAL pgoutput (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:524)
      [2024-01-04 07:24:48,105] INFO [SnapshotModeInitialCrash|task-0] Requested thread factory for connector PostgresConnector, id = snapshot-mode-initial-crash named = SignalProcessor (io.debezium.util.Threads:271)
      [2024-01-04 07:24:48,106] INFO [SnapshotModeInitialCrash|task-0] Requested thread factory for connector PostgresConnector, id = snapshot-mode-initial-crash named = change-event-source-coordinator (io.debezium.util.Threads:271)
      [2024-01-04 07:24:48,106] INFO [SnapshotModeInitialCrash|task-0] Requested thread factory for connector PostgresConnector, id = snapshot-mode-initial-crash named = blocking-snapshot (io.debezium.util.Threads:271)
      [2024-01-04 07:24:48,106] INFO [SnapshotModeInitialCrash|task-0] Creating thread debezium-postgresconnector-snapshot-mode-initial-crash-change-event-source-coordinator (io.debezium.util.Threads:288)
      [2024-01-04 07:24:48,107] INFO [SnapshotModeInitialCrash|task-0] Metrics registered (io.debezium.pipeline.ChangeEventSourceCoordinator:131)
      [2024-01-04 07:24:48,107] INFO [SnapshotModeInitialCrash|task-0] Context created (io.debezium.pipeline.ChangeEventSourceCoordinator:134)
      [2024-01-04 07:24:48,107] INFO [SnapshotModeInitialCrash|task-0] Taking initial snapshot for new datasource (io.debezium.connector.postgresql.snapshot.InitialSnapshotter:34)
      [2024-01-04 07:24:48,107] INFO [SnapshotModeInitialCrash|task-0] According to the connector configuration data will be snapshotted (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:73)
      [2024-01-04 07:24:48,107] INFO [SnapshotModeInitialCrash|task-0] SignalProcessor started. Scheduling it every 5000ms (io.debezium.pipeline.signal.SignalProcessor:105)
      [2024-01-04 07:24:48,108] INFO [SnapshotModeInitialCrash|task-0] Creating thread debezium-postgresconnector-snapshot-mode-initial-crash-SignalProcessor (io.debezium.util.Threads:288)
      [2024-01-04 07:24:48,108] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource:121)
      [2024-01-04 07:24:48,108] INFO [SnapshotModeInitialCrash|task-0] ExactlyOnceWorkerSourceTask{id=SnapshotModeInitialCrash-0} Source task finished initialization and start (org.apache.kafka.connect.runtime.AbstractWorkerSourceTask:275)
      [2024-01-04 07:24:48,109] INFO [SnapshotModeInitialCrash|task-0] Setting isolation level (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:247)
      [2024-01-04 07:24:48,111] INFO [SnapshotModeInitialCrash|task-0] Opening transaction with statement SET TRANSACTION ISOLATION LEVEL REPEATABLE READ;
      SET TRANSACTION SNAPSHOT '0000001A-000000BB-1'; (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:249)
      [2024-01-04 07:24:48,175] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 2 - Determining captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:130)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table framework.dbz_heartbeat to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table meta.version to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table framework.dbz_main_cdc_signal_v001 to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table public.ffl_feature_flags to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table public.dbz_signal to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,177] INFO [SnapshotModeInitialCrash|task-0] Adding table meta.migrations to the list of capture schema tables (io.debezium.relational.RelationalSnapshotChangeEventSource:289)
      [2024-01-04 07:24:48,178] INFO [SnapshotModeInitialCrash|task-0] Created connection pool with 1 threads (io.debezium.relational.RelationalSnapshotChangeEventSource:222)
      [2024-01-04 07:24:48,178] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 3 - Locking captured tables [] (io.debezium.relational.RelationalSnapshotChangeEventSource:139)
      [2024-01-04 07:24:48,178] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 4 - Determining snapshot offset (io.debezium.relational.RelationalSnapshotChangeEventSource:145)
      [2024-01-04 07:24:48,178] INFO [SnapshotModeInitialCrash|task-0] Creating initial offset context (io.debezium.connector.postgresql.PostgresOffsetContext:233)
      [2024-01-04 07:24:48,181] INFO [SnapshotModeInitialCrash|task-0] Read xlogStart at 'LSN{1/5552C4A8}' from transaction '133536' (io.debezium.connector.postgresql.PostgresOffsetContext:236)
      [2024-01-04 07:24:48,182] INFO [SnapshotModeInitialCrash|task-0] Read xlogStart at 'LSN{1/5552C4A8}' from transaction '133536' (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:155)
      [2024-01-04 07:24:48,182] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 5 - Reading structure of captured tables (io.debezium.relational.RelationalSnapshotChangeEventSource:148)
      [2024-01-04 07:24:48,191] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 6 - Persisting schema history (io.debezium.relational.RelationalSnapshotChangeEventSource:152)
      [2024-01-04 07:24:48,191] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 7 - Snapshotting data (io.debezium.relational.RelationalSnapshotChangeEventSource:164)
      [2024-01-04 07:24:48,191] INFO [SnapshotModeInitialCrash|task-0] Creating snapshot worker pool with 1 worker thread(s) (io.debezium.relational.RelationalSnapshotChangeEventSource:413)
      [2024-01-04 07:24:48,194] INFO [SnapshotModeInitialCrash|task-0] Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:104)
      [2024-01-04 07:24:48,194] INFO [SnapshotModeInitialCrash|task-0] Snapshot completed (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:108)
      [2024-01-04 07:24:48,195] INFO [SnapshotModeInitialCrash|task-0] Snapshot ended with SnapshotResult [status=COMPLETED, offset=PostgresOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.postgresql.Source:STRUCT}, sourceInfo=source_info[server='snapshot-mode-initial-crash'db='featureflags', lsn=LSN{1/5552C4A8}, txId=133536, timestamp=2024-01-04T07:24:48.182Z, snapshot=FALSE, schema=, table=], lastSnapshotRecord=true, lastCompletelyProcessedLsn=null, lastCommitLsn=null, streamingStoppingLsn=null, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], incrementalSnapshotContext=IncrementalSnapshotContext [windowOpened=false, chunkEndPosition=null, dataCollectionsToSnapshot=[], lastEventKeySent=null, maximumKey=null]]] (io.debezium.pipeline.ChangeEventSourceCoordinator:254)
      [2024-01-04 07:24:48,195] INFO [SnapshotModeInitialCrash|task-0] Connected metrics set to 'true' (io.debezium.pipeline.ChangeEventSourceCoordinator:423)
      [2024-01-04 07:24:48,207] INFO [SnapshotModeInitialCrash|task-0] REPLICA IDENTITY for 'framework.dbz_main_cdc_signal_v001' is 'FULL'; UPDATE AND DELETE events will contain the previous values of all the columns (io.debezium.connector.postgresql.PostgresSchema:100)
      [2024-01-04 07:24:48,208] INFO [SnapshotModeInitialCrash|task-0] REPLICA IDENTITY for 'public.ffl_feature_flags' is 'FULL'; UPDATE AND DELETE events will contain the previous values of all the columns (io.debezium.connector.postgresql.PostgresSchema:100)
      [2024-01-04 07:24:48,209] INFO [SnapshotModeInitialCrash|task-0] No incremental snapshot in progress, no action needed on start (io.debezium.pipeline.source.snapshot.incremental.AbstractIncrementalSnapshotChangeEventSource:310)
      [2024-01-04 07:24:48,209] INFO [SnapshotModeInitialCrash|task-0] Starting streaming (io.debezium.pipeline.ChangeEventSourceCoordinator:271)
      [2024-01-04 07:24:48,209] INFO [SnapshotModeInitialCrash|task-0] Retrieved latest position from stored offset 'LSN{1/5552C4A8}' (io.debezium.connector.postgresql.PostgresStreamingChangeEventSource:141)
      [2024-01-04 07:24:48,209] INFO [SnapshotModeInitialCrash|task-0] Looking for WAL restart position for last commit LSN 'null' and last change LSN 'LSN{1/5552C4A8}' (io.debezium.connector.postgresql.connection.WalPositionLocator:48)
      [2024-01-04 07:24:48,209] INFO [SnapshotModeInitialCrash|task-0] Initializing PgOutput logical decoder publication (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:150)
      [2024-01-04 07:24:48,321] INFO [SnapshotModeInitialCrash|task-0] Obtained valid replication slot ReplicationSlot [active=false, latestFlushedLsn=LSN{1/5552C4A8}, catalogXmin=133536] (io.debezium.connector.postgresql.connection.PostgresConnection:325)
      [2024-01-04 07:24:48,323] INFO [SnapshotModeInitialCrash|task-0] Connection gracefully closed (io.debezium.jdbc.JdbcConnection:947)
      [2024-01-04 07:24:48,325] INFO [SnapshotModeInitialCrash|task-0] Seeking to LSN{1/5552C4A8} on the replication slot with command SELECT pg_replication_slot_advance('snapshot_mode_initial_crash', '1/5552C4A8') (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:437)
      [2024-01-04 07:24:48,344] INFO [SnapshotModeInitialCrash|task-0] Requested thread factory for connector PostgresConnector, id = snapshot-mode-initial-crash named = keep-alive (io.debezium.util.Threads:271)
      [2024-01-04 07:24:48,345] INFO [SnapshotModeInitialCrash|task-0] Creating thread debezium-postgresconnector-snapshot-mode-initial-crash-keep-alive (io.debezium.util.Threads:288)
      [2024-01-04 07:24:48,365] INFO [SnapshotModeInitialCrash|task-0] REPLICA IDENTITY for 'framework.dbz_main_cdc_signal_v001' is 'FULL'; UPDATE AND DELETE events will contain the previous values of all the columns (io.debezium.connector.postgresql.PostgresSchema:100)
      [2024-01-04 07:24:48,367] INFO [SnapshotModeInitialCrash|task-0] REPLICA IDENTITY for 'public.ffl_feature_flags' is 'FULL'; UPDATE AND DELETE events will contain the previous values of all the columns (io.debezium.connector.postgresql.PostgresSchema:100)
      [2024-01-04 07:24:48,368] INFO [SnapshotModeInitialCrash|task-0] Searching for WAL resume position (io.debezium.connector.postgresql.PostgresStreamingChangeEventSource:341)

      After sending the "take a blocking snapshot" signal, the connector experiences the following error, and does not take any snapshot:

      [2024-01-04 07:30:52,399] INFO [SnapshotModeInitialCrash|task-0] First LSN 'LSN{1/55546288}' received (io.debezium.connector.postgresql.connection.WalPositionLocator:71)
      [2024-01-04 07:30:52,404] INFO [SnapshotModeInitialCrash|task-0] WAL resume position 'LSN{1/55546288}' discovered (io.debezium.connector.postgresql.PostgresStreamingChangeEventSource:362)
      [2024-01-04 07:30:52,417] INFO [SnapshotModeInitialCrash|task-0] Connection gracefully closed (io.debezium.jdbc.JdbcConnection:947)
      [2024-01-04 07:30:52,419] INFO [SnapshotModeInitialCrash|task-0] Connection gracefully closed (io.debezium.jdbc.JdbcConnection:947)
      [2024-01-04 07:30:52,474] INFO [SnapshotModeInitialCrash|task-0] Initializing PgOutput logical decoder publication (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:150)
      [2024-01-04 07:30:52,504] INFO [SnapshotModeInitialCrash|task-0] Seeking to LSN{1/5552C4A8} on the replication slot with command SELECT pg_replication_slot_advance('snapshot_mode_initial_crash', '1/5552C4A8') (io.debezium.connector.postgresql.connection.PostgresReplicationConnection:437)
      [2024-01-04 07:30:52,529] INFO [SnapshotModeInitialCrash|task-0] Requested thread factory for connector PostgresConnector, id = snapshot-mode-initial-crash named = keep-alive (io.debezium.util.Threads:271)
      [2024-01-04 07:30:52,529] INFO [SnapshotModeInitialCrash|task-0] Creating thread debezium-postgresconnector-snapshot-mode-initial-crash-keep-alive (io.debezium.util.Threads:288)
      [2024-01-04 07:30:52,529] INFO [SnapshotModeInitialCrash|task-0] Processing messages (io.debezium.connector.postgresql.PostgresStreamingChangeEventSource:216)
      [2024-01-04 07:30:52,530] INFO [SnapshotModeInitialCrash|task-0] Message with LSN 'LSN{1/55546288}' arrived, switching off the filtering (io.debezium.connector.postgresql.connection.WalPositionLocator:152)
      [2024-01-04 07:30:52,625] INFO [SnapshotModeInitialCrash|task-0] Requested 'BLOCKING' snapshot of data collections '[public.ffl_feature_flags]' with additional conditions '[]' and surrogate key 'PK of table will be used' (io.debezium.pipeline.signal.actions.snapshotting.ExecuteSnapshot:64)
      [2024-01-04 07:30:52,625] INFO [SnapshotModeInitialCrash|task-0] Creating thread debezium-postgresconnector-snapshot-mode-initial-crash-blocking-snapshot (io.debezium.util.Threads:288)
      [2024-01-04 07:30:52,626] INFO [SnapshotModeInitialCrash|task-0] Streaming will now pause (io.debezium.connector.postgresql.PostgresStreamingChangeEventSource:246)
      [2024-01-04 07:30:52,626] INFO [SnapshotModeInitialCrash|task-0] Starting snapshot (io.debezium.pipeline.ChangeEventSourceCoordinator:215)
      [2024-01-04 07:30:52,627] INFO [SnapshotModeInitialCrash|task-0] Snapshot step 1 - Preparing (io.debezium.relational.RelationalSnapshotChangeEventSource:121)
      [2024-01-04 07:30:52,627] INFO [SnapshotModeInitialCrash|task-0] Setting isolation level (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:247)
      [2024-01-04 07:30:52,627] INFO [SnapshotModeInitialCrash|task-0] Opening transaction with statement SET TRANSACTION ISOLATION LEVEL REPEATABLE READ;
      SET TRANSACTION SNAPSHOT '0000001A-000000BB-1'; (io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource:249)
      [2024-01-04 07:30:52,660] ERROR [SnapshotModeInitialCrash|task-0] Error during snapshot (io.debezium.relational.RelationalSnapshotChangeEventSource:179)
      org.postgresql.util.PSQLException: ERROR: invalid snapshot identifier: "0000001A-000000BB-1"
          at org.postgresql.core.v3.QueryExecutorImpl.receiveErrorResponse(QueryExecutorImpl.java:2713)
          at org.postgresql.core.v3.QueryExecutorImpl.processResults(QueryExecutorImpl.java:2401)
          at org.postgresql.core.v3.QueryExecutorImpl.execute(QueryExecutorImpl.java:368)
          at org.postgresql.jdbc.PgStatement.executeInternal(PgStatement.java:498)
          at org.postgresql.jdbc.PgStatement.execute(PgStatement.java:415)
          at org.postgresql.jdbc.PgStatement.executeWithFlags(PgStatement.java:335)
          at org.postgresql.jdbc.PgStatement.executeCachedSql(PgStatement.java:321)
          at org.postgresql.jdbc.PgStatement.executeWithFlags(PgStatement.java:297)
          at org.postgresql.jdbc.PgStatement.execute(PgStatement.java:292)
          at io.debezium.jdbc.JdbcConnection.executeWithoutCommitting(JdbcConnection.java:1449)
          at io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource.setSnapshotTransactionIsolationLevel(PostgresSnapshotChangeEventSource.java:250)
          at io.debezium.connector.postgresql.PostgresSnapshotChangeEventSource.connectionCreated(PostgresSnapshotChangeEventSource.java:99)
          at io.debezium.relational.RelationalSnapshotChangeEventSource.doExecute(RelationalSnapshotChangeEventSource.java:128)
          at io.debezium.pipeline.source.AbstractSnapshotChangeEventSource.execute(AbstractSnapshotChangeEventSource.java:92)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.doSnapshot(ChangeEventSourceCoordinator.java:253)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$doBlockingSnapshot$7(ChangeEventSourceCoordinator.java:218)
          at java.base/java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:515)
          at java.base/java.util.concurrent.FutureTask.run(FutureTask.java:264)
          at java.base/java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1128)
          at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:628)
          at java.base/java.lang.Thread.run(Thread.java:829)
      [2024-01-04 07:30:52,663] INFO [SnapshotModeInitialCrash|task-0] Snapshot - Final stage (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:104)
      [2024-01-04 07:30:52,663] WARN [SnapshotModeInitialCrash|task-0] Snapshot was not completed successfully, it will be re-executed upon connector restart (io.debezium.pipeline.source.AbstractSnapshotChangeEventSource:115)
      [2024-01-04 07:30:52,846] INFO [SnapshotModeInitialCrash|task-0] 1 records sent during previous 00:06:05.976, last recorded offset of {server=snapshot-mode-initial-crash} partition is {transaction_id=null, lsn_proc=5726560904, messageType=INSERT, lsn=5726560904, txId=133577, ts_usec=1704353452217570} (io.debezium.connector.common.BaseSourceTask:215)
      [2024-01-04 07:30:52,977] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 6 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145)
      [2024-01-04 07:30:53,081] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 7 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145)
      [2024-01-04 07:30:53,187] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 8 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145)
      [2024-01-04 07:30:53,291] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 9 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145)
      <snip many more>
      [2024-01-04 07:30:57,841] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 52 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145)
      [2024-01-04 07:30:57,948] WARN [SnapshotModeInitialCrash|task-0] [Producer clientId=connector-producer-SnapshotModeInitialCrash-0, transactionalId=kafka-connect-yoda-SnapshotModeInitialCrash-0] Error while fetching metadata with correlation id 53 : {snapshot-mode-initial-crash.framework.dbz_main_cdc_signal_v001=UNKNOWN_TOPIC_OR_PARTITION} (org.apache.kafka.clients.NetworkClient:1145) 

      Next, I try restarting the connector.  As logged in the similar https://issues.redhat.com/browse/DBZ-7311 bug, the connector fails to gracefully stop, and shows errors/warnings trying to register MBeans that already exist, and takes a lengthy time in a retry loop with "Unable to register metrics as an old set with the same name exists, retrying in PT5S".  A snapshot is never taken upon restart.

      After the restart, if I try inserting the snapshot request into the signal collection table a second time, then a blocking snapshot is taken as normal, as expected.  But there are still numerous warnings like "Error while fetching metadata with correlation id 9 : {snapshot-mode-initial-crash.public.ffl_feature_flags=UNKNOWN_TOPIC_OR_PARTITION}"

      How to reproduce the issue using our tutorial deployment?

      See above for steps to reproduce.  Really all you need to do I think is try to run a blocking ad-hoc snapshot immediately after connector creation whenever snapshot.mode is set to "initial", and no initial table was actually snapshotted.

      Workaround

      I have found that if I proactively restart the connector immediately after creating it with this configuration, then I can subsequently take ad-hoc blocking snapshots for the first time without any issue.  It seems that it is only a freshly-created connector that has yet to restart that is affected by this issue.

              rh-ee-mvitale Mario Fiore Vitale
              james-johnston-thumbtack James Johnston (Inactive)
              Votes:
              0 Vote for this issue
              Watchers:
              5 Start watching this issue

                Created:
                Updated:
                Resolved: