Skip to content

Commit

Permalink
source-postgres-cdc: allow LSN to be acknowledged after sync (#20469)
Browse files Browse the repository at this point in the history
* WIP

* fix issues

* fix test

* format

* more tests

* one more test

* make lsn_commit_behaviour not required

* update expected spec

* update expected spec

* update expected spec

* PR comments

* PR comments

* upgrade version

* auto-bump connector version

Co-authored-by: Octavia Squidington III <octavia-squidington-iii@users.noreply.github.com>
  • Loading branch information
subodh1810 and octavia-squidington-iii committed Jan 4, 2023
1 parent 6a352cf commit 7dfc38d
Show file tree
Hide file tree
Showing 19 changed files with 632 additions and 166 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -1258,7 +1258,7 @@
- name: Postgres
sourceDefinitionId: decd338e-5647-4c0b-adf4-da0e75f5a750
dockerRepository: airbyte/source-postgres
dockerImageTag: 1.0.34
dockerImageTag: 1.0.35
documentationUrl: https://docs.airbyte.com/integrations/sources/postgres
icon: postgresql.svg
sourceType: database
Expand Down
16 changes: 15 additions & 1 deletion airbyte-config/init/src/main/resources/seed/source_specs.yaml
Original file line number Diff line number Diff line change
Expand Up @@ -11364,7 +11364,7 @@
supportsNormalization: false
supportsDBT: false
supported_destination_sync_modes: []
- dockerImage: "airbyte/source-postgres:1.0.34"
- dockerImage: "airbyte/source-postgres:1.0.35"
spec:
documentationUrl: "https://docs.airbyte.com/integrations/sources/postgres"
connectionSpecification:
Expand Down Expand Up @@ -11592,6 +11592,7 @@
- "method"
- "replication_slot"
- "publication"
additionalProperties: true
properties:
method:
type: "string"
Expand Down Expand Up @@ -11636,6 +11637,19 @@
order: 5
min: 120
max: 1200
lsn_commit_behaviour:
type: "string"
title: "LSN commit behaviour"
description: "Determines when Airbtye should flush the LSN of processed\
\ WAL logs in the source database. `After loading Data in the destination`\
\ is default. If `While reading Data` is selected, in case of a\
\ downstream failure (while loading data into the destination),\
\ next sync would result in a full sync."
enum:
- "While reading Data"
- "After loading Data in the destination"
default: "After loading Data in the destination"
order: 6
tunnel_method:
type: "object"
title: "SSH Tunnel Method"
Expand Down
6 changes: 5 additions & 1 deletion airbyte-integrations/bases/debezium-v1-9-6/build.gradle
Original file line number Diff line number Diff line change
Expand Up @@ -13,12 +13,16 @@ dependencies {
implementation 'io.debezium:debezium-embedded:1.9.6.Final'
implementation 'io.debezium:debezium-connector-sqlserver:1.9.6.Final'
implementation 'io.debezium:debezium-connector-mysql:1.9.6.Final'
implementation 'io.debezium:debezium-connector-postgres:1.9.6.Final'
implementation files('debezium-connector-postgres-1.9.6.Final.jar')
implementation 'org.codehaus.plexus:plexus-utils:3.4.2'

testFixturesImplementation project(':airbyte-db:db-lib')
testFixturesImplementation project(':airbyte-integrations:bases:base-java')

testImplementation project(':airbyte-test-utils')
testImplementation libs.connectors.testcontainers.jdbc
testImplementation libs.connectors.testcontainers.postgresql

testFixturesImplementation 'org.junit.jupiter:junit-jupiter-engine:5.4.2'
testFixturesImplementation 'org.junit.jupiter:junit-jupiter-api:5.4.2'
testFixturesImplementation 'org.junit.jupiter:junit-jupiter-params:5.4.2'
Expand Down
Binary file not shown.
Original file line number Diff line number Diff line change
Expand Up @@ -8,6 +8,7 @@
import static io.debezium.connector.postgresql.SourceInfo.LSN_KEY;

import com.fasterxml.jackson.databind.JsonNode;
import io.airbyte.db.jdbc.JdbcUtils;
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog;
import io.debezium.config.Configuration;
import io.debezium.connector.common.OffsetReader;
Expand All @@ -16,10 +17,18 @@
import io.debezium.connector.postgresql.PostgresOffsetContext.Loader;
import io.debezium.connector.postgresql.PostgresPartition;
import io.debezium.connector.postgresql.connection.Lsn;
import io.debezium.jdbc.JdbcConnection.ResultSetMapper;
import io.debezium.jdbc.JdbcConnection.StatementFactory;
import io.debezium.pipeline.spi.Offsets;
import io.debezium.pipeline.spi.Partition;
import java.sql.Connection;
import java.sql.DriverManager;
import java.sql.ResultSet;
import java.sql.SQLException;
import java.sql.Statement;
import java.util.Collections;
import java.util.Map;
import java.util.Objects;
import java.util.Optional;
import java.util.OptionalLong;
import java.util.Properties;
Expand All @@ -29,6 +38,10 @@
import org.apache.kafka.connect.runtime.standalone.StandaloneConfig;
import org.apache.kafka.connect.storage.FileOffsetBackingStore;
import org.apache.kafka.connect.storage.OffsetStorageReaderImpl;
import org.postgresql.core.BaseConnection;
import org.postgresql.replication.LogSequenceNumber;
import org.postgresql.replication.PGReplicationStream;
import org.postgresql.replication.fluent.logical.ChainedLogicalStreamBuilder;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

Expand All @@ -40,33 +53,131 @@ public class PostgresDebeziumStateUtil {

private static final Logger LOGGER = LoggerFactory.getLogger(PostgresDebeziumStateUtil.class);

public boolean isSavedOffsetAfterReplicationSlotLSN(final Properties baseProperties,
final ConfiguredAirbyteCatalog catalog,
final JsonNode cdcState,
final JsonNode replicationSlot,
final JsonNode config) {
public boolean isSavedOffsetAfterReplicationSlotLSN(final JsonNode replicationSlot,
final OptionalLong savedOffset) {

if (Objects.isNull(savedOffset) || savedOffset.isEmpty()) {
return true;
}

if (replicationSlot.has("confirmed_flush_lsn")) {
final long confirmedFlushLsnOnServerSide = Lsn.valueOf(replicationSlot.get("confirmed_flush_lsn").asText()).asLong();
LOGGER.info("Replication slot confirmed_flush_lsn : " + confirmedFlushLsnOnServerSide + " Saved offset LSN : " + savedOffset.getAsLong());
return savedOffset.getAsLong() >= confirmedFlushLsnOnServerSide;
} else if (replicationSlot.has("restart_lsn")) {
final long restartLsn = Lsn.valueOf(replicationSlot.get("restart_lsn").asText()).asLong();
LOGGER.info("Replication slot restart_lsn : " + restartLsn + " Saved offset LSN : " + savedOffset.getAsLong());
return savedOffset.getAsLong() >= restartLsn;
}

// We return true when saved offset is not present cause using an empty offset would result in sync
// from scratch anyway
return true;
}

public OptionalLong savedOffset(final Properties baseProperties,
final ConfiguredAirbyteCatalog catalog,
final JsonNode cdcState,
final JsonNode config) {
final DebeziumPropertiesManager debeziumPropertiesManager = new DebeziumPropertiesManager(baseProperties, config, catalog,
AirbyteFileOffsetBackingStore.initializeState(cdcState),
Optional.empty());
final Properties debeziumProperties = debeziumPropertiesManager.getDebeziumProperties();
final OptionalLong savedOffset = parseSavedOffset(debeziumProperties);

if (savedOffset.isPresent()) {
if (replicationSlot.has("confirmed_flush_lsn")) {
final long confirmedFlushLsnOnServerSide = Lsn.valueOf(replicationSlot.get("confirmed_flush_lsn").asText()).asLong();
LOGGER.info("Replication slot confirmed_flush_lsn : " + confirmedFlushLsnOnServerSide + " Saved offset LSN : " + savedOffset.getAsLong());
return savedOffset.getAsLong() >= confirmedFlushLsnOnServerSide;
} else if (replicationSlot.has("restart_lsn")) {
final long restartLsn = Lsn.valueOf(replicationSlot.get("restart_lsn").asText()).asLong();
LOGGER.info("Replication slot restart_lsn : " + restartLsn + " Saved offset LSN : " + savedOffset.getAsLong());
return savedOffset.getAsLong() >= restartLsn;
return parseSavedOffset(debeziumProperties);
}

private Connection connection(final JsonNode jdbcConfig) throws SQLException {
Properties properties = new Properties();
properties.setProperty("user", jdbcConfig.has(JdbcUtils.USERNAME_KEY) ? jdbcConfig.get(JdbcUtils.USERNAME_KEY).asText()
: null);
properties.setProperty("password", jdbcConfig.has(JdbcUtils.PASSWORD_KEY) ? jdbcConfig.get(JdbcUtils.PASSWORD_KEY).asText()
: null);
properties.setProperty("assumeMinServerVersion", "9.4");
properties.setProperty("ApplicationName", "Airbyte Debezium Streaming");
properties.setProperty("replication", "database");
properties.setProperty("preferQueryMode", "simple"); // replication protocol only supports simple query mode

return DriverManager.getConnection(jdbcConfig.get(JdbcUtils.JDBC_URL_KEY).asText(), properties);
}

public void commitLSNToPostgresDatabase(final JsonNode jdbcConfig,
final OptionalLong savedOffset,
final String slotName,
final String publicationName,
final String plugin) {
if (Objects.isNull(savedOffset) || savedOffset.isEmpty()) {
return;
}

final LogSequenceNumber logSequenceNumber = LogSequenceNumber.valueOf(savedOffset.getAsLong());

try (final BaseConnection pgConnection = ((BaseConnection) connection(jdbcConfig))) {
validateReplicationConnection(pgConnection);

ChainedLogicalStreamBuilder streamBuilder = pgConnection
.getReplicationAPI()
.replicationStream()
.logical()
.withSlotName("\"" + slotName + "\"")
.withStartPosition(logSequenceNumber);

streamBuilder = addSlotOption(publicationName, plugin, pgConnection, streamBuilder);

try (final PGReplicationStream stream = streamBuilder.start()) {
stream.forceUpdateStatus();

stream.setFlushedLSN(logSequenceNumber);
stream.setAppliedLSN(logSequenceNumber);

stream.forceUpdateStatus();
}
} catch (SQLException e) {
throw new RuntimeException(e);
}
}

// We return true when saved offset is not present cause using an empty offset would result in sync
// from scratch anyway
return true;
private ChainedLogicalStreamBuilder addSlotOption(final String publicationName,
final String plugin,
final BaseConnection pgConnection,
ChainedLogicalStreamBuilder streamBuilder) {
if (plugin.equalsIgnoreCase("pgoutput")) {
streamBuilder = streamBuilder.withSlotOption("proto_version", 1)
.withSlotOption("publication_names", publicationName);

if (pgConnection.haveMinimumServerVersion(140000)) {
streamBuilder = streamBuilder.withSlotOption("messages", true);
}
} else if (plugin.equalsIgnoreCase("wal2json")) {
streamBuilder = streamBuilder
.withSlotOption("pretty-print", 1)
.withSlotOption("write-in-chunks", 1)
.withSlotOption("include-xids", 1)
.withSlotOption("include-timestamp", 1)
.withSlotOption("include-not-null", "true");
} else {
throw new RuntimeException("Unknown plugin value : " + plugin);
}
return streamBuilder;
}

private void validateReplicationConnection(final BaseConnection pgConnection) throws SQLException {
final Lsn xlogStart = queryAndMap(pgConnection, "IDENTIFY_SYSTEM", Connection::createStatement, rs -> {
if (!rs.next()) {
throw new IllegalStateException("The DB connection is not a valid replication connection");
}
String xlogpos = rs.getString("xlogpos");
return Lsn.valueOf(xlogpos);
});
}

private <T> T queryAndMap(final Connection conn, final String query, final StatementFactory statementFactory, final ResultSetMapper<T> mapper)
throws SQLException {
Objects.requireNonNull(mapper, "Mapper must be provided");
try (Statement statement = statementFactory.createStatement(conn)) {
try (ResultSet resultSet = statement.executeQuery(query);) {
return mapper.apply(resultSet);
}
}
}

/**
Expand Down
Loading

0 comments on commit 7dfc38d

Please sign in to comment.