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

Message with LSN 'LSN{XYZ}' not present among LSNs seen in the location phase

XMLWordPrintable

    • False
    • None
    • False

      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?

      1.9.6.Final

      What is the connector configuration?

       

      {
              "connector.class": "PostgresCdcSource",
              "database.dbname": "myDB",
              "database.hostname": "10.20.30.40",
              "database.password": "****************",
              "database.port": "5432",
              "database.server.name": "dbserver1",
              "database.sslmode": "require",
              "database.user": "admin",
              "name": "PostgresCdcSource1",
              "output.data.format": "AVRO",
              "plugin.name": "pgoutput",
              "poll.interval.ms": "1000",
              "slot.name": "kafka_slot",
              "snapshot.mode": "exported",
              "table.include.list": "public.myTable",
              "tasks.max": "1"
      }
      

       

      What is the captured database version and mode of depoyment?

      (E.g. on-premises, with a specific cloud provider, etc.)

      Postgres 13 on AWS Cloud

      What behaviour do you expect?

      Connector to process the records without failing.

      What behaviour do you see?

      Connector fails with error message: "Message with LSN 'LSN{XYZ}' not present among LSNs seen in the location phase"

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

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

      Haven't checked with v2.0. This issue is not easily reproducible and is seen for a small set of connectors.

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

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

      INFO Found previous offset PostgresOffsetContext [sourceInfoSchema=Schema{io.debezium.connector.postgresql.Source:STRUCT}, sourceInfo=source_info[server='carhaul_db_prod_metro_logical'db='metro_prod_replica', lsn=LSN\{516/374410E8}, txId=20547285, lastCommitLsn=LSN\{516/374410E8}, timestamp=2022-09-29T06:14:58.864395Z, snapshot=FALSE, schema=, table=], lastSnapshotRecord=false, lastCompletelyProcessedLsn=LSN{516/37448E78}, lastCommitLsn=LSN{516/374410E8}, streamingStoppingLsn=null, transactionContext=TransactionContext [currentTransactionId=null, perTableEventCount={}, totalEventCount=0], incrementalSnapshotContext=IncrementalSnapshotContext [windowOpened=false, chunkEndPosition=null, dataCollectionsToSnapshot=[], lastEventKeySent=null, maximumKey=null]]
      INFO    Found previous partition offset PostgresPartition [sourcePartition=\{server=carhaul_db_prod_metro_logical}]:
      {transaction_id=null, lsn_proc=5592974659192, lsn_commit=5592974627048, lsn=5592974627048, txId=20547285, ts_usec=1664432098864395}
      INFO    Starting streaming
      INFO    Retrieved latest position from stored offset 'LSN{516/37448E78}'INFO    Looking for WAL restart position for last commit LSN 'LSN{516/374410E8}' and last change LSN 'LSN{516/37448E78}'INFO    First LSN 'LSN{0/0}' received
      INFO    LSN after last stored change LSN 'LSN{516/37449180}' received
      INFO    WAL resume position 'LSN{516/37449180}' discovered
      INFO    Streaming requested from LSN LSN{516/37448E78}, received LSN LSN{0/0} identified as already processed
      INFO    Finished streaming
      ERROR   WorkerSourceTask{id=lcc-221p9y-0} Task threw an uncaught and unrecoverable exception. Task is being killed and will not recover until manually restarted
      org.apache.kafka.connect.errors.ConnectException: An exception occurred in the change event producer. This connector will be stopped.
          at io.debezium.pipeline.ErrorHandler.setProducerThrowable(ErrorHandler.java:50)
          at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:180)
          at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:41)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.streamEvents(ChangeEventSourceCoordinator.java:174)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.executeChangeEventSources(ChangeEventSourceCoordinator.java:141)
          at io.debezium.pipeline.ChangeEventSourceCoordinator.lambda$start$0(ChangeEventSourceCoordinator.java:109)
          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:1130)
          at java.base/java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:630)
          at java.base/java.lang.Thread.run(Thread.java:831)
      Caused by: io.debezium.DebeziumException: Message with LSN 'LSN{516/374410E8}' not present among LSNs seen in the location phase '[LSN\{0/0}, LSN\{516/37448E78}, LSN\{516/37448AF0}, LSN\{516/37447738}, LSN\{516/37448848}, LSN\{516/374478E8}, LSN\{516/37447480}, LSN\{516/37441158}, LSN\{516/37442520}, LSN\{516/37449180}, LSN\{516/374489B8}]'. This is unexpected and can lead to an infinite loop or a data loss.
          at io.debezium.connector.postgresql.connection.WalPositionLocator.skipMessage(WalPositionLocator.java:147)
          at io.debezium.connector.postgresql.connection.AbstractMessageDecoder.shouldMessageBeSkipped(AbstractMessageDecoder.java:44)
          at io.debezium.connector.postgresql.connection.pgoutput.PgOutputMessageDecoder.shouldMessageBeSkipped(PgOutputMessageDecoder.java:133)
          at io.debezium.connector.postgresql.connection.PostgresReplicationConnection$1.readPending(PostgresReplicationConnection.java:498)
          at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.processMessages(PostgresStreamingChangeEventSource.java:215)
          at io.debezium.connector.postgresql.PostgresStreamingChangeEventSource.execute(PostgresStreamingChangeEventSource.java:177)
          ... 9 more
      

       

      How to reproduce the issue using our tutorial deployment?

      Not sure.

      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)

      The connector is not processing messages of type ORIGIN or TYPE during the location phase (Searching WAL resume position): https://github.com/debezium/debezium/blob/4f885173018e6c104cf3d0b7afe332e0aaa485a1/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/connection/pgoutput/PgOutputMessageDecoder.java#L208

      And also TRUNCATE messages are skipped conditionally: https://github.com/debezium/debezium/blob/4f885173018e6c104cf3d0b7afe332e0aaa485a1/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/connection/pgoutput/PgOutputMessageDecoder.java#L204

      Due to this the LSN associated with the message of type ORIGIN, TYPE or TRUNCATE is not being stored in the lsnSeen set.
      But while processing these messages in the second phase, we are trying to find them in the lsnSeen set which are not there: https://github.com/debezium/debezium/blob/4f885173018e6c104cf3d0b7afe332e0aaa485a1/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/connection/pgoutput/PgOutputMessageDecoder.java#L133

            Unassigned Unassigned
            rdangwal Rajendra Dangwal
            Votes:
            0 Vote for this issue
            Watchers:
            5 Start watching this issue

              Created:
              Updated:
              Resolved: