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

Graceful shutdown not working when Postgres LSNs need flushing

XMLWordPrintable

    • Icon: Bug Bug
    • Resolution: Unresolved
    • Icon: Major Major
    • 1.9-backlog
    • 1.4.2.Final
    • embedded-engine
    • None

      There is an ordering issue with graceful shutdown on Postgres databases using Embedded engine. The replication stream is closed before flush.

      The following things happen when EmbeddedEngine.close() is called:

      1. The PostgresStreamingChangeEventSource.execute(..) method is terminated and the following finally block is triggered.
        See LOC https://github.com/debezium/debezium/blob/master/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java#L152-L173
        public void execute(ChangeEventSourceContext context) throws InterruptedException {
          ...
                finally {
                    if (replicationConnection != null) {
                       ...
                       replicationStream.set(null);
                    }
            }
        }
        
      2. In EmbeddedEngine the finally block of the run method is invoked and this code block is executed:
        See LOC:  https://github.com/debezium/debezium/blob/86cb71ad8b063a66fff8148a9a2dc94830d21d09/debezium-embedded/src/main/java/io/debezium/embedded/EmbeddedEngine.java#L838-L850
                            finally {
                                    ...
                                    // Always commit offsets that were captured from the source records we actually processed ...
                                    commitOffsets(offsetWriter, commitTimeout, task);
                                    ...
                                }
        
      3. This triggers the execution of the PostgresStreamingChangeEventSource commitOffset method, but replicationStream was set to null in step 1, so the flushLsn never happens:
        See LOC: https://github.com/debezium/debezium/blob/master/debezium-connector-postgres/src/main/java/io/debezium/connector/postgresql/PostgresStreamingChangeEventSource.java#L334-L347
            @Override
            public void commitOffset(Map<String, ?> offset) {
                try {
                    ...
                    if (replicationStream != null && lsn != null) {
                       ...
                        // tell the server the point up to which we've processed data, so it can be free to recycle WAL segments
                        replicationStream.flushLsn(lsn);
                    }
                    ...
            }
        
      4. The shutdown sequence completes with no exceptions raised, despite the complete failure to flush LSNs

            Unassigned Unassigned
            cjbooms Conor Gallagher (Inactive)
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

              Created:
              Updated: