diff --git a/airbyte-commons/src/main/java/io/airbyte/commons/io/IOs.java b/airbyte-commons/src/main/java/io/airbyte/commons/io/IOs.java index 91f140be7760..4f0833bbc980 100644 --- a/airbyte-commons/src/main/java/io/airbyte/commons/io/IOs.java +++ b/airbyte-commons/src/main/java/io/airbyte/commons/io/IOs.java @@ -48,6 +48,15 @@ public static Path writeFile(Path path, String fileName, String contents) { return writeFile(filePath, contents); } + public static Path writeFile(Path filePath, byte[] contents) { + try { + Files.write(filePath, contents); + return filePath; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + public static Path writeFile(Path filePath, String contents) { try { Files.writeString(filePath, contents, StandardCharsets.UTF_8); diff --git a/airbyte-commons/src/main/java/io/airbyte/commons/resources/MoreResources.java b/airbyte-commons/src/main/java/io/airbyte/commons/resources/MoreResources.java index 4eae26a439b9..145b98cd2874 100644 --- a/airbyte-commons/src/main/java/io/airbyte/commons/resources/MoreResources.java +++ b/airbyte-commons/src/main/java/io/airbyte/commons/resources/MoreResources.java @@ -46,6 +46,11 @@ public static String readResource(String name) throws IOException { return Resources.toString(resource, StandardCharsets.UTF_8); } + public static byte[] readBytes(String name) throws IOException { + URL resource = Resources.getResource(name); + return Resources.toByteArray(resource); + } + /** * This class is a bit of a hack. Might have unexpected behavior. * diff --git a/airbyte-commons/src/main/java/io/airbyte/commons/util/AutoCloseableIterators.java b/airbyte-commons/src/main/java/io/airbyte/commons/util/AutoCloseableIterators.java index c58eb82c45b1..31d969184d2b 100644 --- a/airbyte-commons/src/main/java/io/airbyte/commons/util/AutoCloseableIterators.java +++ b/airbyte-commons/src/main/java/io/airbyte/commons/util/AutoCloseableIterators.java @@ -72,6 +72,15 @@ public static AutoCloseableIterator fromStream(Stream stream) { return new DefaultAutoCloseableIterator<>(stream.iterator(), stream::close); } + /** + * Consumes entire iterator and collect it into a list. Then it closes the iterator. + */ + public static List toListAndClose(AutoCloseableIterator iterator) throws Exception { + try (iterator) { + return MoreIterators.toList(iterator); + } + } + /** * Returns a {@link AutoCloseableIterator} that will call the provided supplier ONE time when * {@link AutoCloseableIterator#hasNext()} is called the first time. The supplier returns a stream @@ -131,6 +140,11 @@ public static AutoCloseableIterator transform(Function(iteratorCreator.apply(autoCloseableIterator), autoCloseableIterator::close); } + @SafeVarargs + public static CompositeIterator concatWithEagerClose(AutoCloseableIterator... iterators) { + return concatWithEagerClose(List.of(iterators)); + } + public static CompositeIterator concatWithEagerClose(List> iterators) { return new CompositeIterator<>(iterators); } diff --git a/airbyte-commons/src/main/java/io/airbyte/commons/util/MoreIterators.java b/airbyte-commons/src/main/java/io/airbyte/commons/util/MoreIterators.java index c01dc840e268..a341b7e3bc12 100644 --- a/airbyte-commons/src/main/java/io/airbyte/commons/util/MoreIterators.java +++ b/airbyte-commons/src/main/java/io/airbyte/commons/util/MoreIterators.java @@ -24,12 +24,14 @@ package io.airbyte.commons.util; +import com.google.common.collect.AbstractIterator; import java.util.ArrayList; import java.util.Arrays; import java.util.HashSet; import java.util.Iterator; import java.util.List; import java.util.Set; +import java.util.function.Supplier; public class MoreIterators { @@ -74,4 +76,22 @@ public static Set toSet(Iterator iterator) { return set; } + public static Iterator singletonIteratorFromSupplier(Supplier supplier) { + return new AbstractIterator() { + + private boolean hasSupplied = false; + + @Override + protected T computeNext() { + if (!hasSupplied) { + hasSupplied = true; + return supplier.get(); + } else { + return endOfData(); + } + } + + }; + } + } diff --git a/airbyte-commons/src/main/resources/log4j2.xml b/airbyte-commons/src/main/resources/log4j2.xml index 6858a2a3ae62..423500072af0 100644 --- a/airbyte-commons/src/main/resources/log4j2.xml +++ b/airbyte-commons/src/main/resources/log4j2.xml @@ -50,7 +50,7 @@ - + diff --git a/airbyte-commons/src/test/java/io/airbyte/commons/io/IOsTest.java b/airbyte-commons/src/test/java/io/airbyte/commons/io/IOsTest.java index 03600d20c5fa..23ed08a74b99 100644 --- a/airbyte-commons/src/test/java/io/airbyte/commons/io/IOsTest.java +++ b/airbyte-commons/src/test/java/io/airbyte/commons/io/IOsTest.java @@ -34,6 +34,7 @@ import java.io.FileWriter; import java.io.IOException; import java.io.Writer; +import java.nio.charset.StandardCharsets; import java.nio.file.Files; import java.nio.file.Path; import java.util.Collections; @@ -55,6 +56,16 @@ void testReadWrite() throws IOException { assertEquals("abc", IOs.readFile(path.resolve("file"))); } + @Test + void testWriteBytes() throws IOException { + final Path path = Files.createTempDirectory("tmp"); + + final Path filePath = IOs.writeFile(path.resolve("file"), "abc".getBytes(StandardCharsets.UTF_8)); + + assertEquals(path.resolve("file"), filePath); + assertEquals("abc", IOs.readFile(path, "file")); + } + @Test public void testWriteFileToRandomDir() throws IOException { final String contents = "something to remember"; diff --git a/airbyte-commons/src/test/java/io/airbyte/commons/resources/MoreResourcesTest.java b/airbyte-commons/src/test/java/io/airbyte/commons/resources/MoreResourcesTest.java index 555d3ad04801..793c88935d5d 100644 --- a/airbyte-commons/src/test/java/io/airbyte/commons/resources/MoreResourcesTest.java +++ b/airbyte-commons/src/test/java/io/airbyte/commons/resources/MoreResourcesTest.java @@ -29,6 +29,7 @@ import com.google.common.collect.Sets; import java.io.IOException; +import java.nio.charset.StandardCharsets; import java.nio.file.Path; import java.util.stream.Collectors; import org.junit.jupiter.api.Test; @@ -43,6 +44,14 @@ void testResourceRead() throws IOException { assertThrows(IllegalArgumentException.class, () -> MoreResources.readResource("invalid")); } + @Test + void testReadBytes() throws IOException { + assertEquals("content1\n", new String(MoreResources.readBytes("resource_test"), StandardCharsets.UTF_8)); + assertEquals("content2\n", new String(MoreResources.readBytes("subdir/resource_test_sub"), StandardCharsets.UTF_8)); + + assertThrows(IllegalArgumentException.class, () -> MoreResources.readBytes("invalid")); + } + @Test void testResourceReadDuplicateName() throws IOException { assertEquals("content1\n", MoreResources.readResource("resource_test_a")); diff --git a/airbyte-config/init/src/main/resources/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json b/airbyte-config/init/src/main/resources/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json index 67236503ae09..ff914ac82d25 100644 --- a/airbyte-config/init/src/main/resources/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json +++ b/airbyte-config/init/src/main/resources/config/STANDARD_SOURCE_DEFINITION/decd338e-5647-4c0b-adf4-da0e75f5a750.json @@ -2,6 +2,6 @@ "sourceDefinitionId": "decd338e-5647-4c0b-adf4-da0e75f5a750", "name": "Postgres", "dockerRepository": "airbyte/source-postgres", - "dockerImageTag": "0.2.3", + "dockerImageTag": "0.2.4", "documentationUrl": "https://hub.docker.com/r/airbyte/source-postgres" } diff --git a/airbyte-config/init/src/main/resources/seed/source_definitions.yaml b/airbyte-config/init/src/main/resources/seed/source_definitions.yaml index 1811d6f4815c..f0e8d123dcb7 100644 --- a/airbyte-config/init/src/main/resources/seed/source_definitions.yaml +++ b/airbyte-config/init/src/main/resources/seed/source_definitions.yaml @@ -26,7 +26,7 @@ - sourceDefinitionId: decd338e-5647-4c0b-adf4-da0e75f5a750 name: Postgres dockerRepository: airbyte/source-postgres - dockerImageTag: 0.2.3 + dockerImageTag: 0.2.4 documentationUrl: https://hub.docker.com/r/airbyte/source-postgres - sourceDefinitionId: cd42861b-01fc-4658-a8ab-5d11d0510f01 name: Recurly diff --git a/airbyte-db/src/main/java/io/airbyte/db/PgLsn.java b/airbyte-db/src/main/java/io/airbyte/db/PgLsn.java new file mode 100644 index 000000000000..75ae56810a47 --- /dev/null +++ b/airbyte-db/src/main/java/io/airbyte/db/PgLsn.java @@ -0,0 +1,104 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.db; + +import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; +import java.nio.ByteBuffer; + +/** + * Doc on the structure of a Postgres LSN + * https://www.postgresql.org/docs/current/datatype-pg-lsn.html + */ +public class PgLsn implements Comparable { + + private final long lsn; + + public static PgLsn fromLong(final long lsn) { + return new PgLsn(lsn); + } + + public static PgLsn fromPgString(final String lsn) { + return new PgLsn(lsnToLong(lsn)); + } + + private PgLsn(final long lsn) { + this.lsn = lsn; + } + + public long asLong() { + return lsn; + } + + public String asPgString() { + return longToLsn(lsn); + } + + @Override + public int compareTo(final PgLsn o) { + return Long.compare(lsn, o.asLong()); + } + + /** + * The LSN returned by Postgres is a 64-bit integer represented as hex encoded 32-bit integers + * separated by a /. reference: https://github.com/davecramer/LogicalDecode + * + * @param lsn string representation as returned by postgres + * @return long representation of the lsn string. + */ + @VisibleForTesting + static long lsnToLong(String lsn) { + int slashIndex = lsn.lastIndexOf('/'); + Preconditions.checkArgument(slashIndex >= 0); + + String logicalXLogStr = lsn.substring(0, slashIndex); + // parses as a long but then cast to int. this allows us to retain the full 32 bits of the integer + // as opposed to the reduced value of Integer.MAX_VALUE. + int logicalXlog = (int) Long.parseLong(logicalXLogStr, 16); + String segmentStr = lsn.substring(slashIndex + 1, lsn.length()); + int segment = (int) Long.parseLong(segmentStr, 16); + + ByteBuffer buf = ByteBuffer.allocate(8); + buf.putInt(logicalXlog); + buf.putInt(segment); + buf.position(0); + return buf.getLong(); + } + + @VisibleForTesting + static String longToLsn(long long1) { + int front = (int) (long1 >> 32); + int back = (int) long1; + return (Integer.toHexString(front) + "/" + Integer.toHexString(back)).toUpperCase(); + } + + @Override + public String toString() { + return "PgLsn{" + + "lsn=" + lsn + + '}'; + } + +} diff --git a/airbyte-db/src/main/java/io/airbyte/db/PostgresUtils.java b/airbyte-db/src/main/java/io/airbyte/db/PostgresUtils.java new file mode 100644 index 000000000000..0a39e1a34626 --- /dev/null +++ b/airbyte-db/src/main/java/io/airbyte/db/PostgresUtils.java @@ -0,0 +1,45 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.db; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.base.Preconditions; +import io.airbyte.db.jdbc.JdbcDatabase; +import io.airbyte.db.jdbc.JdbcUtils; +import java.sql.SQLException; +import java.util.List; + +public class PostgresUtils { + + public static PgLsn getLsn(JdbcDatabase database) throws SQLException { + // pg version 10+. + final List jsonNodes = database + .bufferedResultSetQuery(conn -> conn.createStatement().executeQuery("SELECT pg_current_wal_lsn()"), JdbcUtils::rowToJson); + + Preconditions.checkState(jsonNodes.size() == 1); + return PgLsn.fromPgString(jsonNodes.get(0).get("pg_current_wal_lsn").asText()); + } + +} diff --git a/airbyte-db/src/test/java/io/airbyte/db/PgLsnTest.java b/airbyte-db/src/test/java/io/airbyte/db/PgLsnTest.java new file mode 100644 index 000000000000..a13261fe0768 --- /dev/null +++ b/airbyte-db/src/test/java/io/airbyte/db/PgLsnTest.java @@ -0,0 +1,56 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.db; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +import com.google.common.collect.ImmutableMap; +import java.util.Map; +import org.junit.jupiter.api.Test; + +class PgLsnTest { + + private static final Map TEST_LSNS = ImmutableMap.builder() + .put("0/15E7A10", 22968848L) + .put("0/15E7B08", 22969096L) + .put("16/15E7B08", 94512249608L) + .put("16/FFFFFFFF", 98784247807L) + .put("7FFFFFFF/FFFFFFFF", Long.MAX_VALUE) + .put("0/0", 0L) + .build(); + + @Test + void testLsnToLong() { + TEST_LSNS.forEach( + (key, value) -> assertEquals(value, PgLsn.lsnToLong(key), String.format("Conversion failed. lsn: %s long value: %s", key, value))); + } + + @Test + void testLongToLsn() { + TEST_LSNS.forEach( + (key, value) -> assertEquals(key, PgLsn.longToLsn(value), String.format("Conversion failed. lsn: %s long value: %s", key, value))); + } + +} diff --git a/airbyte-db/src/test/java/io/airbyte/db/PostgresUtilsTest.java b/airbyte-db/src/test/java/io/airbyte/db/PostgresUtilsTest.java new file mode 100644 index 000000000000..6fb77d215637 --- /dev/null +++ b/airbyte-db/src/test/java/io/airbyte/db/PostgresUtilsTest.java @@ -0,0 +1,115 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.db; + +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.io.IOs; +import io.airbyte.commons.json.Jsons; +import io.airbyte.db.jdbc.DefaultJdbcDatabase; +import io.airbyte.db.jdbc.JdbcDatabase; +import io.airbyte.test.utils.PostgreSQLContainerHelper; +import java.sql.SQLException; +import org.apache.commons.dbcp2.BasicDataSource; +import org.apache.commons.lang3.RandomStringUtils; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.utility.MountableFile; + +class PostgresUtilsTest { + + private static PostgreSQLContainer PSQL_DB; + + private BasicDataSource dataSource; + + @BeforeAll + static void init() { + PSQL_DB = new PostgreSQLContainer<>("postgres:13-alpine"); + PSQL_DB.start(); + + } + + @BeforeEach + void setup() throws Exception { + final String dbName = "db_" + RandomStringUtils.randomAlphabetic(10).toLowerCase(); + + final JsonNode config = getConfig(PSQL_DB, dbName); + + final String initScriptName = "init_" + dbName.concat(".sql"); + final String tmpFilePath = IOs.writeFileToRandomTmpDir(initScriptName, "CREATE DATABASE " + dbName + ";"); + PostgreSQLContainerHelper.runSqlScript(MountableFile.forHostPath(tmpFilePath), PSQL_DB); + + dataSource = new BasicDataSource(); + dataSource.setDriverClassName("org.postgresql.Driver"); + dataSource.setUsername(config.get("username").asText()); + dataSource.setPassword(config.get("password").asText()); + dataSource.setUrl(String.format("jdbc:postgresql://%s:%s/%s", + config.get("host").asText(), + config.get("port").asText(), + config.get("database").asText())); + + final JdbcDatabase defaultJdbcDatabase = new DefaultJdbcDatabase(dataSource); + + defaultJdbcDatabase.execute(connection -> { + connection.createStatement().execute("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));"); + connection.createStatement().execute("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');"); + }); + } + + private JsonNode getConfig(PostgreSQLContainer psqlDb, String dbName) { + return Jsons.jsonNode(ImmutableMap.builder() + .put("host", psqlDb.getHost()) + .put("port", psqlDb.getFirstMappedPort()) + .put("database", dbName) + .put("username", psqlDb.getUsername()) + .put("password", psqlDb.getPassword()) + .build()); + } + + @Test + void testGetLsn() throws SQLException { + final JdbcDatabase database = new DefaultJdbcDatabase(dataSource); + + final PgLsn lsn1 = PostgresUtils.getLsn(database); + assertNotNull(lsn1); + assertTrue(lsn1.asLong() > 0); + + database.execute(connection -> { + connection.createStatement().execute("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');"); + }); + + final PgLsn lsn2 = PostgresUtils.getLsn(database); + assertNotNull(lsn2); + assertTrue(lsn2.asLong() > 0); + + assertTrue(lsn1.compareTo(lsn2) < 0, "returned lsns are not ascending."); + } + +} diff --git a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/StandardSourceTest.java b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/StandardSourceTest.java index 8389b0e83bc5..7823d0231778 100644 --- a/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/StandardSourceTest.java +++ b/airbyte-integrations/bases/standard-source-test/src/main/java/io/airbyte/integrations/standardtest/source/StandardSourceTest.java @@ -32,6 +32,7 @@ import static org.junit.jupiter.api.Assertions.assertTrue; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.collect.Sets; import io.airbyte.commons.json.Jsons; import io.airbyte.config.JobGetSpecConfig; @@ -75,6 +76,10 @@ public abstract class StandardSourceTest { + public static final String CDC_LSN = "_ab_cdc_lsn"; + public static final String CDC_UPDATED_AT = "_ab_cdc_updated_at"; + public static final String CDC_DELETED_AT = "_ab_cdc_deleted_at"; + private static final long JOB_ID = 0L; private static final int JOB_ATTEMPT = 0; @@ -446,7 +451,11 @@ private List runRead(ConfiguredAirbyteCatalog catalog, JsonNode private void assertSameRecords(List expected, List actual, String message) { final List prunedExpected = expected.stream().map(this::pruneEmittedAt).collect(Collectors.toList()); - final List prunedActual = actual.stream().map(this::pruneEmittedAt).collect(Collectors.toList()); + final List prunedActual = actual + .stream() + .map(this::pruneEmittedAt) + .map(this::pruneCdcMetadata) + .collect(Collectors.toList()); assertEquals(prunedExpected.size(), prunedActual.size(), message); assertTrue(prunedExpected.containsAll(prunedActual), message); assertTrue(prunedActual.containsAll(prunedExpected), message); @@ -456,6 +465,14 @@ private AirbyteRecordMessage pruneEmittedAt(AirbyteRecordMessage m) { return Jsons.clone(m).withEmittedAt(null); } + private AirbyteRecordMessage pruneCdcMetadata(AirbyteRecordMessage m) { + final AirbyteRecordMessage clone = Jsons.clone(m); + ((ObjectNode) clone.getData()).remove(CDC_LSN); + ((ObjectNode) clone.getData()).remove(CDC_UPDATED_AT); + ((ObjectNode) clone.getData()).remove(CDC_DELETED_AT); + return clone; + } + public static class TestDestinationEnv { private final Path localRoot; diff --git a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/AbstractJdbcSource.java b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/AbstractJdbcSource.java index 19fcd7ff95a1..1410be20b79b 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/AbstractJdbcSource.java +++ b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/AbstractJdbcSource.java @@ -26,8 +26,10 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; +import io.airbyte.commons.functional.CheckedConsumer; import io.airbyte.commons.json.Jsons; import io.airbyte.commons.lang.Exceptions; import io.airbyte.commons.util.AutoCloseableIterator; @@ -79,6 +81,10 @@ public abstract class AbstractJdbcSource extends BaseConnector implements Source private static final Logger LOGGER = LoggerFactory.getLogger(AbstractJdbcSource.class); + public static final String CDC_LSN = "_ab_cdc_lsn"; + public static final String CDC_UPDATED_AT = "_ab_cdc_updated_at"; + public static final String CDC_DELETED_AT = "_ab_cdc_deleted_at"; + private static final String JDBC_COLUMN_DATABASE_NAME = "TABLE_CAT"; private static final String JDBC_COLUMN_SCHEMA_NAME = "TABLE_SCHEM"; private static final String JDBC_COLUMN_TABLE_NAME = "TABLE_NAME"; @@ -118,18 +124,31 @@ public AbstractJdbcSource(final String driverClass, final JdbcStreamingQueryConf @Override public AirbyteConnectionStatus check(JsonNode config) { try (final JdbcDatabase database = createDatabase(config)) { - // attempt to get metadata from the database as a cheap way of seeing if we can connect. - database.bufferedResultSetQuery(conn -> conn.getMetaData().getCatalogs(), JdbcUtils::rowToJson); + for (CheckedConsumer checkOperation : getCheckOperations(config)) { + checkOperation.accept(database); + } return new AirbyteConnectionStatus().withStatus(Status.SUCCEEDED); } catch (Exception e) { - LOGGER.debug("Exception while checking connection: ", e); + LOGGER.info("Exception while checking connection: ", e); return new AirbyteConnectionStatus() .withStatus(Status.FAILED) .withMessage("Could not connect with provided configuration."); } } + /** + * Configures a list of operations that can be used to check the connection to the source. + * + * @return list of consumers that run queries for the check command. + */ + public List> getCheckOperations(JsonNode config) throws Exception { + return ImmutableList.of(database -> { + LOGGER.info("Attempting to get metadata from the database to see if we can connect."); + database.bufferedResultSetQuery(conn -> conn.getMetaData().getCatalogs(), JdbcUtils::rowToJson); + }); + } + @Override public AirbyteCatalog discover(JsonNode config) throws Exception { try (final JdbcDatabase database = createDatabase(config)) { @@ -149,8 +168,9 @@ public AirbyteCatalog discover(JsonNode config) throws Exception { @Override public AutoCloseableIterator read(JsonNode config, ConfiguredAirbyteCatalog catalog, JsonNode state) throws Exception { - final JdbcStateManager stateManager = - new JdbcStateManager(state == null ? JdbcStateManager.emptyState() : Jsons.object(state, JdbcState.class), catalog); + final JdbcStateManager stateManager = new JdbcStateManager( + state == null ? JdbcStateManager.emptyState() : Jsons.object(state, JdbcState.class), + catalog); final Instant emittedAt = Instant.now(); final JdbcDatabase database = createDatabase(config); @@ -161,9 +181,9 @@ public AutoCloseableIterator read(JsonNode config, ConfiguredAir .collect(Collectors.toMap(t -> String.format("%s.%s", t.getSchemaName(), t.getName()), Function.identity())); final List> incrementalIterators = - getIncrementalIterators(database, catalog, tableNameToTable, stateManager, emittedAt); + getIncrementalIterators(config, database, catalog, tableNameToTable, stateManager, emittedAt); final List> fullRefreshIterators = - getFullRefreshIterators(database, catalog, tableNameToTable, stateManager, emittedAt); + getFullRefreshIterators(config, database, catalog, tableNameToTable, stateManager, emittedAt); final List> iteratorList = Stream.of(incrementalIterators, fullRefreshIterators) .flatMap(Collection::stream) .collect(Collectors.toList()); @@ -175,7 +195,8 @@ public AutoCloseableIterator read(JsonNode config, ConfiguredAir }); } - public List> getIncrementalIterators(JdbcDatabase database, + public List> getIncrementalIterators(JsonNode config, + JdbcDatabase database, ConfiguredAirbyteCatalog catalog, Map tableNameToTable, JdbcStateManager stateManager, @@ -189,7 +210,8 @@ public List> getIncrementalIterators(JdbcD configuredStream -> configuredStream.getSyncMode().equals(SyncMode.INCREMENTAL)); } - public List> getFullRefreshIterators(JdbcDatabase database, + public List> getFullRefreshIterators(JsonNode config, + JdbcDatabase database, ConfiguredAirbyteCatalog catalog, Map tableNameToTable, JdbcStateManager stateManager, @@ -471,9 +493,9 @@ private List discoverInternal(final JdbcDatabase database, fi .collect(Collectors.toList()); } - private static AutoCloseableIterator getMessageIterator(AutoCloseableIterator recordIterator, - String streamName, - long emittedAt) { + public static AutoCloseableIterator getMessageIterator(AutoCloseableIterator recordIterator, + String streamName, + long emittedAt) { return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessage() .withType(Type.RECORD) .withRecord(new AirbyteRecordMessage() @@ -589,7 +611,7 @@ public List getPrimaryKeys() { * The following two classes are internal data structures to ease managing tables. Any external * information should be revealed through the {@link TableInfo} class. */ - static class TableInfoInternal { + public static class TableInfoInternal { private final String schemaName; private final String name; diff --git a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcCdcStateManager.java b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcCdcStateManager.java new file mode 100644 index 000000000000..9461c1bfbd24 --- /dev/null +++ b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcCdcStateManager.java @@ -0,0 +1,65 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.jdbc; + +import com.google.common.annotations.VisibleForTesting; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.jdbc.models.CdcState; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class JdbcCdcStateManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(JdbcStateManager.class); + + private final CdcState initialState; + + private CdcState currentState; + + @VisibleForTesting + JdbcCdcStateManager(CdcState serialized) { + this.initialState = serialized; + this.currentState = serialized; + + LOGGER.info("Initialized CDC state with: {}", serialized); + } + + public void setCdcState(CdcState state) { + this.currentState = state; + } + + public CdcState getCdcState() { + return currentState != null ? Jsons.clone(currentState) : null; + } + + @Override + public String toString() { + return "JdbcCdcStateManager{" + + "initialState=" + initialState + + ", currentState=" + currentState + + '}'; + } + +} diff --git a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcStateManager.java b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcStateManager.java index 31b2810a4283..5460c2a7bc79 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcStateManager.java +++ b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/JdbcStateManager.java @@ -54,12 +54,16 @@ public class JdbcStateManager { private static final Logger LOGGER = LoggerFactory.getLogger(JdbcStateManager.class); private final Map streamNameToCursorInfo; + private Boolean isCdc; + private final JdbcCdcStateManager cdcStateManager; public static JdbcState emptyState() { return new JdbcState(); } public JdbcStateManager(JdbcState serialized, ConfiguredAirbyteCatalog catalog) { + this.cdcStateManager = new JdbcCdcStateManager(serialized.getCdcState()); + this.isCdc = serialized.getCdc(); streamNameToCursorInfo = new ImmutableMap.Builder().putAll(createCursorInfoMap(serialized, catalog)).build(); } @@ -160,22 +164,43 @@ public Optional getCursor(String streamName) { } synchronized public AirbyteStateMessage updateAndEmit(String streamName, String cursor) { - final Optional cursorInfo = getCursorInfo(streamName); - Preconditions.checkState(cursorInfo.isPresent(), "Could not find cursor information for stream: " + streamName); - cursorInfo.get().setCursor(cursor); + // cdc file gets updated by debezium so the "update" part is a no op. + if (!isCdc) { + final Optional cursorInfo = getCursorInfo(streamName); + Preconditions.checkState(cursorInfo.isPresent(), "Could not find cursor information for stream: " + streamName); + cursorInfo.get().setCursor(cursor); + } + + return toState(); + } + + public void setIsCdc(boolean isCdc) { + if (this.isCdc == null) { + this.isCdc = isCdc; + } else { + Preconditions.checkState(this.isCdc == isCdc, "attempt to set cdc to {}, but is already set to {}.", isCdc, this.isCdc); + } + } + + public JdbcCdcStateManager getCdcStateManager() { + return cdcStateManager; + } + public AirbyteStateMessage emit() { return toState(); } private AirbyteStateMessage toState() { final JdbcState jdbcState = new JdbcState() + .withCdc(isCdc) .withStreams(streamNameToCursorInfo.entrySet().stream() .sorted(Entry.comparingByKey()) // sort by stream name for sanity. .map(e -> new JdbcStreamState() .withStreamName(e.getKey()) .withCursorField(e.getValue().getCursorField() == null ? Collections.emptyList() : Lists.newArrayList(e.getValue().getCursorField())) .withCursor(e.getValue().getCursor())) - .collect(Collectors.toList())); + .collect(Collectors.toList())) + .withCdcState(cdcStateManager.getCdcState()); return new AirbyteStateMessage().withData(Jsons.jsonNode(jdbcState)); } diff --git a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/StateDecoratingIterator.java b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/StateDecoratingIterator.java index 4a3306bf80a4..a7255dcb8bd2 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/StateDecoratingIterator.java +++ b/airbyte-integrations/connectors/source-jdbc/src/main/java/io/airbyte/integrations/source/jdbc/StateDecoratingIterator.java @@ -33,7 +33,7 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -class StateDecoratingIterator extends AbstractIterator implements Iterator { +public class StateDecoratingIterator extends AbstractIterator implements Iterator { private static final Logger LOGGER = LoggerFactory.getLogger(StateDecoratingIterator.class); @@ -58,6 +58,7 @@ public StateDecoratingIterator(Iterator messageIterator, this.cursorField = cursorField; this.cursorType = cursorType; this.maxCursor = initialCursor; + stateManager.setIsCdc(false); } @Override diff --git a/airbyte-integrations/connectors/source-jdbc/src/main/resources/jdbc_models/jdbc_models.yaml b/airbyte-integrations/connectors/source-jdbc/src/main/resources/jdbc_models/jdbc_models.yaml index d248e712b4e3..11da2ec29bd0 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/main/resources/jdbc_models/jdbc_models.yaml +++ b/airbyte-integrations/connectors/source-jdbc/src/main/resources/jdbc_models/jdbc_models.yaml @@ -10,10 +10,16 @@ properties: definitions: JdbcState: type: object - additionalProperties: false - required: - - streams properties: + cdc: + type: boolean + cdc_state: + type: object + properties: + state: + description: json representation of debezium state. + type: object + existingJavaType: com.fasterxml.jackson.databind.JsonNode streams: description: "State for each stream." type: array diff --git a/airbyte-integrations/connectors/source-jdbc/src/test/java/io/airbyte/integrations/source/jdbc/JdbcStateManagerTest.java b/airbyte-integrations/connectors/source-jdbc/src/test/java/io/airbyte/integrations/source/jdbc/JdbcStateManagerTest.java index d78bdc479ff0..b5f1e462a937 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/test/java/io/airbyte/integrations/source/jdbc/JdbcStateManagerTest.java +++ b/airbyte-integrations/connectors/source-jdbc/src/test/java/io/airbyte/integrations/source/jdbc/JdbcStateManagerTest.java @@ -28,6 +28,7 @@ import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.source.jdbc.JdbcStateManager.CursorInfo; +import io.airbyte.integrations.source.jdbc.models.CdcState; import io.airbyte.integrations.source.jdbc.models.JdbcState; import io.airbyte.integrations.source.jdbc.models.JdbcStreamState; import io.airbyte.protocol.models.AirbyteStateMessage; @@ -155,7 +156,9 @@ void testToState() { new JdbcStreamState().withStreamName(STREAM_NAME1).withCursorField(Lists.newArrayList(CURSOR_FIELD1)).withCursor("a"), new JdbcStreamState().withStreamName(STREAM_NAME2).withCursorField(Lists.newArrayList(CURSOR_FIELD2)), new JdbcStreamState().withStreamName(STREAM_NAME3)) - .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())))); + .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())) + .withCdc(false) + .withCdcState(new CdcState()))); final AirbyteStateMessage actualFirstEmission = stateManager.updateAndEmit(STREAM_NAME1, "a"); assertEquals(expectedFirstEmission, actualFirstEmission); final AirbyteStateMessage expectedSecondEmission = new AirbyteStateMessage() @@ -163,7 +166,9 @@ void testToState() { .newArrayList(new JdbcStreamState().withStreamName(STREAM_NAME1).withCursorField(Lists.newArrayList(CURSOR_FIELD1)).withCursor("a"), new JdbcStreamState().withStreamName(STREAM_NAME2).withCursorField(Lists.newArrayList(CURSOR_FIELD2)).withCursor("b"), new JdbcStreamState().withStreamName(STREAM_NAME3)) - .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())))); + .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())) + .withCdc(false) + .withCdcState(new CdcState()))); final AirbyteStateMessage actualSecondEmission = stateManager.updateAndEmit(STREAM_NAME2, "b"); assertEquals(expectedSecondEmission, actualSecondEmission); } @@ -184,7 +189,9 @@ void testToStateNullCursorField() { .newArrayList( new JdbcStreamState().withStreamName(STREAM_NAME1).withCursorField(Lists.newArrayList(CURSOR_FIELD1)).withCursor("a"), new JdbcStreamState().withStreamName(STREAM_NAME2)) - .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())))); + .stream().sorted(Comparator.comparing(JdbcStreamState::getStreamName)).collect(Collectors.toList())) + .withCdc(false) + .withCdcState(new CdcState()))); final AirbyteStateMessage actualFirstEmission = stateManager.updateAndEmit(STREAM_NAME1, "a"); assertEquals(expectedFirstEmission, actualFirstEmission); diff --git a/airbyte-integrations/connectors/source-jdbc/src/testFixtures/java/io/airbyte/integrations/source/jdbc/test/JdbcSourceStandardTest.java b/airbyte-integrations/connectors/source-jdbc/src/testFixtures/java/io/airbyte/integrations/source/jdbc/test/JdbcSourceStandardTest.java index c2def0ce253e..028b7ba3bee9 100644 --- a/airbyte-integrations/connectors/source-jdbc/src/testFixtures/java/io/airbyte/integrations/source/jdbc/test/JdbcSourceStandardTest.java +++ b/airbyte-integrations/connectors/source-jdbc/src/testFixtures/java/io/airbyte/integrations/source/jdbc/test/JdbcSourceStandardTest.java @@ -547,17 +547,20 @@ void testReadOneTableIncrementallyTwice() throws Exception { expectedMessages.add(new AirbyteMessage().withType(Type.RECORD) .withRecord(new AirbyteRecordMessage().withStream(streamName) .withData(Jsons.jsonNode(ImmutableMap - .of(COL_ID, ID_VALUE_4, COL_NAME, "riker", COL_UPDATED_AT, - "2006-10-19T00:00:00Z"))))); + .of(COL_ID, ID_VALUE_4, + COL_NAME, "riker", + COL_UPDATED_AT, "2006-10-19T00:00:00Z"))))); expectedMessages.add(new AirbyteMessage().withType(Type.RECORD) .withRecord(new AirbyteRecordMessage().withStream(streamName) .withData(Jsons.jsonNode(ImmutableMap - .of(COL_ID, ID_VALUE_5, COL_NAME, "data", COL_UPDATED_AT, - "2006-10-19T00:00:00Z"))))); + .of(COL_ID, ID_VALUE_5, + COL_NAME, "data", + COL_UPDATED_AT, "2006-10-19T00:00:00Z"))))); expectedMessages.add(new AirbyteMessage() .withType(Type.STATE) .withState(new AirbyteStateMessage() .withData(Jsons.jsonNode(new JdbcState() + .withCdc(false) .withStreams(Lists.newArrayList(new JdbcStreamState() .withStreamName(streamName) .withCursorField(ImmutableList.of(COL_ID)) @@ -626,6 +629,7 @@ void testReadMultipleTablesIncrementally() throws Exception { .withType(Type.STATE) .withState(new AirbyteStateMessage() .withData(Jsons.jsonNode(new JdbcState() + .withCdc(false) .withStreams(Lists.newArrayList( new JdbcStreamState() .withStreamName(streamName) @@ -634,11 +638,13 @@ void testReadMultipleTablesIncrementally() throws Exception { new JdbcStreamState() .withStreamName(streamName2) .withCursorField(ImmutableList.of(COL_ID)))))))); + expectedMessagesFirstSync.addAll(secondStreamExpectedMessages); expectedMessagesFirstSync.add(new AirbyteMessage() .withType(Type.STATE) .withState(new AirbyteStateMessage() .withData(Jsons.jsonNode(new JdbcState() + .withCdc(false) .withStreams(Lists.newArrayList( new JdbcStreamState() .withStreamName(streamName) @@ -648,6 +654,7 @@ void testReadMultipleTablesIncrementally() throws Exception { .withStreamName(streamName2) .withCursorField(ImmutableList.of(COL_ID)) .withCursor("3"))))))); + setEmittedAtToNull(actualMessagesFirstSync); assertEquals(expectedMessagesFirstSync, actualMessagesFirstSync); @@ -707,6 +714,7 @@ private void incrementalCursorCheck( .withType(Type.STATE) .withState(new AirbyteStateMessage() .withData(Jsons.jsonNode(new JdbcState() + .withCdc(false) .withStreams(Lists.newArrayList(new JdbcStreamState() .withStreamName(airbyteStream.getStream().getName()) .withCursorField(ImmutableList.of(cursorField)) @@ -761,18 +769,22 @@ private List getTestMessages() { new AirbyteMessage().withType(Type.RECORD) .withRecord(new AirbyteRecordMessage().withStream(streamName) .withData(Jsons.jsonNode(ImmutableMap - .of(COL_ID, ID_VALUE_1, COL_NAME, "picard", COL_UPDATED_AT, - "2004-10-19T00:00:00Z")))), + .of(COL_ID, ID_VALUE_1, + COL_NAME, "picard", + COL_UPDATED_AT, "2004-10-19T00:00:00Z")))), new AirbyteMessage().withType(Type.RECORD) .withRecord(new AirbyteRecordMessage().withStream(streamName) .withData(Jsons.jsonNode(ImmutableMap - .of(COL_ID, ID_VALUE_2, COL_NAME, "crusher", COL_UPDATED_AT, + .of(COL_ID, ID_VALUE_2, + COL_NAME, "crusher", + COL_UPDATED_AT, "2005-10-19T00:00:00Z")))), new AirbyteMessage().withType(Type.RECORD) .withRecord(new AirbyteRecordMessage().withStream(streamName) .withData(Jsons.jsonNode(ImmutableMap - .of(COL_ID, ID_VALUE_3, COL_NAME, "vash", COL_UPDATED_AT, - "2006-10-19T00:00:00Z"))))); + .of(COL_ID, ID_VALUE_3, + COL_NAME, "vash", + COL_UPDATED_AT, "2006-10-19T00:00:00Z"))))); } private ConfiguredAirbyteStream createTableWithSpaces() throws SQLException { diff --git a/airbyte-integrations/connectors/source-postgres/Dockerfile b/airbyte-integrations/connectors/source-postgres/Dockerfile index 20f32aabb7e3..1699e92068d9 100644 --- a/airbyte-integrations/connectors/source-postgres/Dockerfile +++ b/airbyte-integrations/connectors/source-postgres/Dockerfile @@ -8,5 +8,5 @@ COPY build/distributions/${APPLICATION}*.tar ${APPLICATION}.tar RUN tar xf ${APPLICATION}.tar --strip-components=1 -LABEL io.airbyte.version=0.2.3 +LABEL io.airbyte.version=0.2.4 LABEL io.airbyte.name=airbyte/source-postgres diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index b9b45d4e00e7..65dc6edee667 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -14,12 +14,16 @@ dependencies { implementation project(':airbyte-protocol:models') implementation project(':airbyte-integrations:connectors:source-jdbc') + implementation 'org.apache.commons:commons-lang3:3.11' implementation "org.postgresql:postgresql:42.2.18" + implementation 'io.debezium:debezium-embedded:1.4.2.Final' + implementation 'io.debezium:debezium-api:1.4.2.Final' + implementation 'io.debezium:debezium-connector-postgres:1.4.2.Final' testImplementation testFixtures(project(':airbyte-integrations:connectors:source-jdbc')) + testImplementation project(":airbyte-json-validation") testImplementation project(':airbyte-test-utils') - testImplementation 'org.apache.commons:commons-lang3:3.11' testImplementation 'org.testcontainers:postgresql:1.15.1' integrationTestJavaImplementation project(':airbyte-integrations:bases:standard-source-test') diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStore.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStore.java new file mode 100644 index 000000000000..0976db4cbe3e --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStore.java @@ -0,0 +1,152 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.base.Preconditions; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.jdbc.models.CdcState; +import java.io.EOFException; +import java.io.IOException; +import java.io.ObjectOutputStream; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.NoSuchFileException; +import java.nio.file.Path; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.stream.Collectors; +import org.apache.commons.io.FileUtils; +import org.apache.kafka.connect.errors.ConnectException; +import org.apache.kafka.connect.util.SafeObjectInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class handles reading and writing a debezium offset file. In many cases it is duplicating + * logic in debezium because that logic is not exposed in the public API. We mostly treat the + * contents of this state file like a black box. We know it is a Map. We + * deserialize it to a Map so that the state file can be human readable. If we ever + * discover that any of the contents of these offset files is not string serializable we will likely + * have to drop the human readability support and just base64 encode it. + */ +public class AirbyteFileOffsetBackingStore { + + private static final Logger LOGGER = LoggerFactory.getLogger(AirbyteFileOffsetBackingStore.class); + + private final Path offsetFilePath; + + public AirbyteFileOffsetBackingStore(final Path offsetFilePath) { + this.offsetFilePath = offsetFilePath; + } + + public Path getOffsetFilePath() { + return offsetFilePath; + } + + public CdcState read() { + final Map raw = load(); + + final Map mappedAsStrings = raw.entrySet().stream().collect(Collectors.toMap( + e -> byteBufferToString(e.getKey()), + e -> byteBufferToString(e.getValue()))); + final JsonNode asJson = Jsons.jsonNode(mappedAsStrings); + + LOGGER.info("debezium state: {}", asJson); + + return new CdcState().withState(asJson); + } + + @SuppressWarnings("unchecked") + public void persist(CdcState cdcState) { + final Map mapAsString = + cdcState != null && cdcState.getState() != null ? Jsons.object(cdcState.getState(), Map.class) : Collections.emptyMap(); + final Map mappedAsStrings = mapAsString.entrySet().stream().collect(Collectors.toMap( + e -> stringToByteBuffer(e.getKey()), + e -> stringToByteBuffer(e.getValue()))); + + FileUtils.deleteQuietly(offsetFilePath.toFile()); + save(mappedAsStrings); + } + + private static String byteBufferToString(ByteBuffer byteBuffer) { + Preconditions.checkNotNull(byteBuffer); + return new String(byteBuffer.array(), StandardCharsets.UTF_8); + } + + private static ByteBuffer stringToByteBuffer(String s) { + Preconditions.checkNotNull(s); + return ByteBuffer.wrap(s.getBytes(StandardCharsets.UTF_8)); + } + + /** + * See FileOffsetBackingStore#load - logic is mostly borrowed from here. duplicated because this + * method is not public. + */ + @SuppressWarnings("unchecked") + private Map load() { + try (final SafeObjectInputStream is = new SafeObjectInputStream(Files.newInputStream(offsetFilePath))) { + final Object obj = is.readObject(); + if (!(obj instanceof HashMap)) + throw new ConnectException("Expected HashMap but found " + obj.getClass()); + final Map raw = (Map) obj; + final Map data = new HashMap<>(); + for (Map.Entry mapEntry : raw.entrySet()) { + final ByteBuffer key = (mapEntry.getKey() != null) ? ByteBuffer.wrap(mapEntry.getKey()) : null; + final ByteBuffer value = (mapEntry.getValue() != null) ? ByteBuffer.wrap(mapEntry.getValue()) : null; + data.put(key, value); + } + + return data; + } catch (NoSuchFileException | EOFException e) { + // NoSuchFileException: Ignore, may be new. + // EOFException: Ignore, this means the file was missing or corrupt + return Collections.emptyMap(); + } catch (IOException | ClassNotFoundException e) { + throw new ConnectException(e); + } + } + + /** + * See FileOffsetBackingStore#save - logic is mostly borrowed from here. duplicated because this + * method is not public. + */ + private void save(Map data) { + try (ObjectOutputStream os = new ObjectOutputStream(Files.newOutputStream(offsetFilePath))) { + Map raw = new HashMap<>(); + for (Map.Entry mapEntry : data.entrySet()) { + byte[] key = (mapEntry.getKey() != null) ? mapEntry.getKey().array() : null; + byte[] value = (mapEntry.getValue() != null) ? mapEntry.getValue().array() : null; + raw.put(key, value); + } + os.writeObject(raw); + } catch (IOException e) { + throw new ConnectException(e); + } + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumEventUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumEventUtils.java new file mode 100644 index 000000000000..9a5f2bcd5e97 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumEventUtils.java @@ -0,0 +1,77 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.airbyte.commons.json.Jsons; +import io.airbyte.protocol.models.AirbyteMessage; +import io.airbyte.protocol.models.AirbyteRecordMessage; +import io.debezium.engine.ChangeEvent; +import java.time.Instant; + +public class DebeziumEventUtils { + + public static AirbyteMessage toAirbyteMessage(ChangeEvent event, Instant emittedAt) { + final JsonNode debeziumRecord = Jsons.deserialize(event.value()); + final JsonNode before = debeziumRecord.get("before"); + final JsonNode after = debeziumRecord.get("after"); + final JsonNode source = debeziumRecord.get("source"); + final String op = debeziumRecord.get("op").asText(); + + final JsonNode data = formatDebeziumData(before, after, source); + + final String streamName = source.get("schema").asText() + "." + source.get("table").asText(); + + final AirbyteRecordMessage airbyteRecordMessage = new AirbyteRecordMessage() + .withStream(streamName) + .withEmittedAt(emittedAt.toEpochMilli()) + .withData(data); + + return new AirbyteMessage() + .withType(AirbyteMessage.Type.RECORD) + .withRecord(airbyteRecordMessage); + } + + // warning mutates input args. + private static JsonNode formatDebeziumData(JsonNode before, JsonNode after, JsonNode source) { + final ObjectNode base = (ObjectNode) (after.isNull() ? before : after); + + long transactionMillis = source.get("ts_ms").asLong(); + long lsn = source.get("lsn").asLong(); + + base.put("_ab_cdc_updated_at", transactionMillis); + base.put("_ab_cdc_lsn", lsn); + + if (after.isNull()) { + base.put("_ab_cdc_deleted_at", transactionMillis); + } else { + base.put("_ab_cdc_deleted_at", (Long) null); + } + + return base; + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordIterator.java new file mode 100644 index 000000000000..4b33ab655b66 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordIterator.java @@ -0,0 +1,168 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import com.google.common.collect.AbstractIterator; +import io.airbyte.commons.concurrency.VoidCallable; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.lang.MoreBooleans; +import io.airbyte.commons.util.AutoCloseableIterator; +import io.airbyte.db.PgLsn; +import io.debezium.engine.ChangeEvent; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.Optional; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.function.Supplier; +import org.apache.kafka.connect.data.Struct; +import org.apache.kafka.connect.source.SourceRecord; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The record iterator is the consumer (in the producer / consumer relationship with debezium) is + * responsible for 1. making sure every record produced by the record publisher is processed 2. + * signalling to the record publisher when it is time for it to stop producing records. It emits + * this signal either when the publisher had not produced a new record for a long time or when it + * has processed at least all of the records that were present in the database when the source was + * started. Because the publisher might publish more records between the consumer sending this + * signal and the publisher acutally shutting down, the consumer must stay alive as long as the + * publisher is not closed or if there are any new records for it to process (even if the publisher + * is closed). + */ +public class DebeziumRecordIterator extends AbstractIterator> + implements AutoCloseableIterator> { + + private static final Logger LOGGER = LoggerFactory.getLogger(DebeziumRecordIterator.class); + + private static final TimeUnit SLEEP_TIME_UNIT = TimeUnit.SECONDS; + private static final int SLEEP_TIME_AMOUNT = 5; + + private final LinkedBlockingQueue> queue; + private final PgLsn targetLsn; + private final Supplier publisherStatusSupplier; + private final VoidCallable requestClose; + + public DebeziumRecordIterator(LinkedBlockingQueue> queue, + PgLsn targetLsn, + Supplier publisherStatusSupplier, + VoidCallable requestClose) { + this.queue = queue; + this.targetLsn = targetLsn; + this.publisherStatusSupplier = publisherStatusSupplier; + this.requestClose = requestClose; + } + + @Override + protected ChangeEvent computeNext() { + // keep trying until the publisher is closed or until the queue is empty. the latter case is + // possible when the publisher has shutdown but the consumer has not yet processed all messages it + // emitted. + while (!MoreBooleans.isTruthy(publisherStatusSupplier.get()) || !queue.isEmpty()) { + final ChangeEvent next; + try { + next = queue.poll(SLEEP_TIME_AMOUNT, SLEEP_TIME_UNIT); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + + // if within the timeout, the consumer could not get a record, it is time to tell the producer to + // shutdown. + if (next == null) { + requestClose(); + LOGGER.info("no record found. polling again."); + continue; + } + + // if the last record matches the target lsn, it is time to tell the producer to shutdown. + if (shouldSignalClose(next)) { + requestClose(); + } + + return next; + } + return endOfData(); + } + + @Override + public void close() throws Exception { + requestClose.call(); + } + + private boolean shouldSignalClose(ChangeEvent event) { + final PgLsn eventLsn = extractLsn(event); + + if (targetLsn.compareTo(eventLsn) > 0) { + return false; + } else { + final SnapshotMetadata snapshotMetadata = getSnapshotMetadata(event); + // if not snapshot or is snapshot but last record in snapshot. + return SnapshotMetadata.TRUE != snapshotMetadata; + } + } + + private SnapshotMetadata getSnapshotMetadata(ChangeEvent event) { + try { + final Method sourceRecordMethod = event.getClass().getMethod("sourceRecord"); + sourceRecordMethod.setAccessible(true); + final SourceRecord sourceRecord = (SourceRecord) sourceRecordMethod.invoke(event); + final String snapshot = ((Struct) sourceRecord.value()).getStruct("source").getString("snapshot"); + + if (snapshot == null) { + return null; + } + + // the snapshot field is an enum of true, false, and last. + return SnapshotMetadata.valueOf(snapshot.toUpperCase()); + } catch (NoSuchMethodException | IllegalAccessException | InvocationTargetException e) { + throw new RuntimeException(e); + } + } + + private PgLsn extractLsn(ChangeEvent event) { + return Optional.ofNullable(event.value()) + .flatMap(value -> Optional.ofNullable(Jsons.deserialize(value).get("source"))) + .flatMap(source -> Optional.ofNullable(source.get("lsn").asText())) + .map(Long::parseLong) + .map(PgLsn::fromLong) + .orElseThrow(() -> new IllegalStateException("Could not find LSN")); + } + + private void requestClose() { + try { + requestClose.call(); + } catch (Exception e) { + throw new RuntimeException(e); + } + } + + enum SnapshotMetadata { + TRUE, + FALSE, + LAST + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisher.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisher.java new file mode 100644 index 000000000000..b7f894c1b10d --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisher.java @@ -0,0 +1,182 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.annotations.VisibleForTesting; +import io.airbyte.protocol.models.AirbyteStream; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.SyncMode; +import io.debezium.engine.ChangeEvent; +import io.debezium.engine.DebeziumEngine; +import io.debezium.engine.format.Json; +import io.debezium.engine.spi.OffsetCommitPolicy; +import java.util.Properties; +import java.util.Queue; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import org.codehaus.plexus.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class DebeziumRecordPublisher implements AutoCloseable { + + private static final Logger LOGGER = LoggerFactory.getLogger(DebeziumRecordPublisher.class); + private final ExecutorService executor; + private DebeziumEngine> engine; + + private final JsonNode config; + private final ConfiguredAirbyteCatalog catalog; + private final AirbyteFileOffsetBackingStore offsetManager; + + private final AtomicBoolean hasClosed; + private final AtomicBoolean isClosing; + private final AtomicReference thrownError; + private final CountDownLatch engineLatch; + + public DebeziumRecordPublisher(JsonNode config, ConfiguredAirbyteCatalog catalog, AirbyteFileOffsetBackingStore offsetManager) { + this.config = config; + this.catalog = catalog; + this.offsetManager = offsetManager; + this.hasClosed = new AtomicBoolean(false); + this.isClosing = new AtomicBoolean(false); + this.thrownError = new AtomicReference<>(); + this.executor = Executors.newSingleThreadExecutor(); + this.engineLatch = new CountDownLatch(1); + } + + public void start(Queue> queue) { + engine = DebeziumEngine.create(Json.class) + .using(getDebeziumProperties(config, catalog, offsetManager)) + .using(new OffsetCommitPolicy.AlwaysCommitOffsetPolicy()) + .notifying(e -> { + // debezium outputs a tombstone event that has a value of null. this is an artifact of how it + // interacts with kafka. we want to ignore it. + // more on the tombstone: + // https://debezium.io/documentation/reference/configuration/event-flattening.html + if (e.value() != null) { + queue.add(e); + } + }) + .using((success, message, error) -> { + LOGGER.info("Debezium engine shutdown."); + thrownError.set(error); + engineLatch.countDown(); + }) + .build(); + + // Run the engine asynchronously ... + executor.execute(engine); + } + + public boolean hasClosed() { + return hasClosed.get(); + } + + public void close() throws Exception { + if (isClosing.compareAndSet(false, true)) { + // consumers should assume records can be produced until engine has closed. + if (engine != null) { + engine.close(); + } + + // wait for closure before shutting down executor service + engineLatch.await(5, TimeUnit.MINUTES); + + // shut down and await for thread to actually go down + executor.shutdown(); + executor.awaitTermination(5, TimeUnit.MINUTES); + + // after the engine is completely off, we can mark this as closed + hasClosed.set(true); + + if (thrownError.get() != null) { + throw new RuntimeException(thrownError.get()); + } + } + } + + protected static Properties getDebeziumProperties(JsonNode config, ConfiguredAirbyteCatalog catalog, AirbyteFileOffsetBackingStore offsetManager) { + final Properties props = new Properties(); + + // debezium engine configuration + props.setProperty("name", "engine"); + props.setProperty("plugin.name", "pgoutput"); + props.setProperty("connector.class", "io.debezium.connector.postgresql.PostgresConnector"); + props.setProperty("offset.storage", "org.apache.kafka.connect.storage.FileOffsetBackingStore"); + props.setProperty("offset.storage.file.filename", offsetManager.getOffsetFilePath().toString()); + props.setProperty("offset.flush.interval.ms", "1000"); // todo: make this longer + props.setProperty("snapshot.mode", "exported"); + + // https://debezium.io/documentation/reference/configuration/avro.html + props.setProperty("key.converter.schemas.enable", "false"); + props.setProperty("value.converter.schemas.enable", "false"); + + // debezium names + props.setProperty("name", config.get("database").asText()); + props.setProperty("database.server.name", config.get("database").asText()); + + // db connection configuration + props.setProperty("database.hostname", config.get("host").asText()); + props.setProperty("database.port", config.get("port").asText()); + props.setProperty("database.user", config.get("username").asText()); + props.setProperty("database.dbname", config.get("database").asText()); + + if (config.has("password")) { + props.setProperty("database.password", config.get("password").asText()); + } + + props.setProperty("slot.name", config.get("replication_method").get("replication_slot").asText()); + props.setProperty("publication.name", config.get("replication_method").get("publication").asText()); + + // table selection + final String tableWhitelist = getTableWhitelist(catalog); + props.setProperty("table.include.list", tableWhitelist); + props.setProperty("database.include.list", config.get("database").asText()); + + // recommended when using pgoutput + props.setProperty("publication.autocreate.mode", "disabled"); + + return props; + } + + @VisibleForTesting + protected static String getTableWhitelist(ConfiguredAirbyteCatalog catalog) { + return catalog.getStreams().stream() + .filter(s -> s.getSyncMode() == SyncMode.INCREMENTAL) + .map(ConfiguredAirbyteStream::getStream) + .map(AirbyteStream::getName) + // debezium needs commas escaped to split properly + .map(x -> StringUtils.escape(x, new char[] {','}, "\\,")) + .collect(Collectors.joining(",")); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresSource.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresSource.java index 2b1a13d693c6..652dba68e17f 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresSource.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresSource.java @@ -24,14 +24,50 @@ package io.airbyte.integrations.source.postgres; +import static java.util.stream.Collectors.toList; + import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableMap; +import io.airbyte.commons.functional.CheckedConsumer; import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.util.AutoCloseableIterator; +import io.airbyte.commons.util.AutoCloseableIterators; +import io.airbyte.commons.util.CompositeIterator; +import io.airbyte.commons.util.MoreIterators; +import io.airbyte.db.PgLsn; +import io.airbyte.db.PostgresUtils; +import io.airbyte.db.jdbc.JdbcDatabase; +import io.airbyte.db.jdbc.JdbcUtils; import io.airbyte.db.jdbc.PostgresJdbcStreamingQueryConfiguration; import io.airbyte.integrations.base.IntegrationRunner; import io.airbyte.integrations.base.Source; import io.airbyte.integrations.source.jdbc.AbstractJdbcSource; +import io.airbyte.integrations.source.jdbc.JdbcStateManager; +import io.airbyte.protocol.models.AirbyteCatalog; +import io.airbyte.protocol.models.AirbyteConnectionStatus; +import io.airbyte.protocol.models.AirbyteMessage; +import io.airbyte.protocol.models.AirbyteMessage.Type; +import io.airbyte.protocol.models.AirbyteStateMessage; +import io.airbyte.protocol.models.AirbyteStream; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.SyncMode; +import io.debezium.engine.ChangeEvent; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; import java.util.Set; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.function.Supplier; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -66,6 +102,196 @@ public Set getExcludedInternalSchemas() { return Set.of("information_schema", "pg_catalog", "pg_internal", "catalog_history"); } + @Override + public AirbyteCatalog discover(JsonNode config) throws Exception { + AirbyteCatalog catalog = super.discover(config); + + if (isCdc(config)) { + final List streams = catalog.getStreams().stream() + .map(PostgresSource::removeIncrementalWithoutPk) + .map(PostgresSource::addCdcMetadataColumns) + .collect(toList()); + + catalog.setStreams(streams); + } + + return catalog; + } + + @Override + public List> getCheckOperations(JsonNode config) throws Exception { + final List> checkOperations = new ArrayList<>(super.getCheckOperations(config)); + + if (isCdc(config)) { + checkOperations.add(database -> { + List matchingSlots = database.query(connection -> { + final String sql = "SELECT * FROM pg_replication_slots WHERE slot_name = ? AND plugin = ? AND database = ?"; + PreparedStatement ps = connection.prepareStatement(sql); + ps.setString(1, config.get("replication_method").get("replication_slot").asText()); + ps.setString(2, "pgoutput"); + ps.setString(3, config.get("database").asText()); + + LOGGER.info("Attempting to find the named replication slot using the query: " + ps.toString()); + + return ps; + }, JdbcUtils::rowToJson).collect(toList()); + + if (matchingSlots.size() != 1) { + throw new RuntimeException("Expected exactly one replication slot but found " + matchingSlots.size() + + ". Please read the docs and add a replication slot to your database."); + } + + }); + + checkOperations.add(database -> { + List matchingPublications = database.query(connection -> { + PreparedStatement ps = connection.prepareStatement("SELECT * FROM pg_publication WHERE pubname = ?"); + ps.setString(1, config.get("replication_method").get("publication").asText()); + + LOGGER.info("Attempting to find the publication using the query: " + ps.toString()); + + return ps; + }, JdbcUtils::rowToJson).collect(toList()); + + if (matchingPublications.size() != 1) { + throw new RuntimeException("Expected exactly one publication but found " + matchingPublications.size() + + ". Please read the docs and add a publication to your database."); + } + + }); + } + + return checkOperations; + } + + @Override + public AutoCloseableIterator read(JsonNode config, ConfiguredAirbyteCatalog catalog, JsonNode state) throws Exception { + // this check is used to ensure that have the pgoutput slot available so Debezium won't attempt to + // create it. + final AirbyteConnectionStatus check = check(config); + + if (check.getStatus().equals(AirbyteConnectionStatus.Status.FAILED)) { + throw new RuntimeException("Unable establish a connection: " + check.getMessage()); + } + + return super.read(config, catalog, state); + } + + private static PgLsn getLsn(JdbcDatabase database) { + try { + return PostgresUtils.getLsn(database); + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + private AirbyteFileOffsetBackingStore initializeState(JdbcStateManager stateManager) { + final Path cdcWorkingDir; + try { + cdcWorkingDir = Files.createTempDirectory(Path.of("/tmp"), "cdc"); + } catch (IOException e) { + throw new RuntimeException(e); + } + final Path cdcOffsetFilePath = cdcWorkingDir.resolve("offset.dat"); + + final AirbyteFileOffsetBackingStore offsetManager = new AirbyteFileOffsetBackingStore(cdcOffsetFilePath); + offsetManager.persist(stateManager.getCdcStateManager().getCdcState()); + return offsetManager; + } + + @Override + public List> getIncrementalIterators(JsonNode config, + JdbcDatabase database, + ConfiguredAirbyteCatalog catalog, + Map tableNameToTable, + JdbcStateManager stateManager, + Instant emittedAt) { + if (isCdc(config)) { + // State works differently in CDC than it does in convention incremental. The state is written to an + // offset file that debezium reads from. Then once all records are replicated, we read back that + // offset file (which will have been updated by debezium) and set it in the state. There is no + // incremental updating of the state structs in the CDC impl. + final AirbyteFileOffsetBackingStore offsetManager = initializeState(stateManager); + + final PgLsn targetLsn = getLsn(database); + LOGGER.info("identified target lsn: " + targetLsn); + + final LinkedBlockingQueue> queue = new LinkedBlockingQueue<>(); + + final DebeziumRecordPublisher publisher = new DebeziumRecordPublisher(config, catalog, offsetManager); + publisher.start(queue); + + // handle state machine around pub/sub logic. + final AutoCloseableIterator> eventIterator = new DebeziumRecordIterator( + queue, + targetLsn, + publisher::hasClosed, + publisher::close); + + // convert to airbyte message. + final AutoCloseableIterator messageIterator = AutoCloseableIterators.transform( + eventIterator, + (event) -> DebeziumEventUtils.toAirbyteMessage(event, emittedAt)); + + // our goal is to get the state at the time this supplier is called (i.e. after all message records + // have been produced) + final Supplier stateMessageSupplier = () -> { + stateManager.getCdcStateManager().setCdcState(offsetManager.read()); + final AirbyteStateMessage stateMessage = stateManager.emit(); + return new AirbyteMessage().withType(Type.STATE).withState(stateMessage); + }; + + // wrap the supplier in an iterator so that we can concat it to the message iterator. + final Iterator stateMessageIterator = MoreIterators.singletonIteratorFromSupplier(stateMessageSupplier); + + // this structure guarantees that the debezium engine will be closed, before we attempt to emit the + // state file. we want this so that we have a guarantee that the debezium offset file (which we use + // to produce the state file) is up-to-date. + final CompositeIterator messageIteratorWithStateDecorator = AutoCloseableIterators + .concatWithEagerClose(messageIterator, AutoCloseableIterators.fromIterator(stateMessageIterator)); + + return Collections.singletonList(messageIteratorWithStateDecorator); + } else { + return super.getIncrementalIterators(config, database, catalog, tableNameToTable, stateManager, emittedAt); + } + } + + @VisibleForTesting + static boolean isCdc(JsonNode config) { + LOGGER.info("isCdc config: " + config); + final boolean isCdc = config.hasNonNull("replication_method") + && config.get("replication_method").hasNonNull("replication_slot") + && config.get("replication_method").hasNonNull("publication"); + LOGGER.info("using CDC: {}", isCdc); + return isCdc; + } + + /* + * It isn't possible to recreate the state of the original database unless we include extra + * information (like an oid) when using logical replication. By limiting to Full Refresh when we + * don't have a primary key we dodge the problem for now. As a work around a CDC and non-CDC source + * could be configured if there's a need to replicate a large non-PK table. + */ + private static AirbyteStream removeIncrementalWithoutPk(AirbyteStream stream) { + if (stream.getSourceDefinedPrimaryKey().isEmpty()) { + stream.getSupportedSyncModes().remove(SyncMode.INCREMENTAL); + } + + return stream; + } + + private static AirbyteStream addCdcMetadataColumns(AirbyteStream stream) { + ObjectNode jsonSchema = (ObjectNode) stream.getJsonSchema(); + ObjectNode properties = (ObjectNode) jsonSchema.get("properties"); + + final JsonNode numberType = Jsons.jsonNode(ImmutableMap.of("type", "number")); + properties.set(CDC_LSN, numberType); + properties.set(CDC_UPDATED_AT, numberType); + properties.set(CDC_DELETED_AT, numberType); + + return stream; + } + public static void main(String[] args) throws Exception { final Source source = new PostgresSource(); LOGGER.info("starting source: {}", PostgresSource.class); diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/postgresql.conf b/airbyte-integrations/connectors/source-postgres/src/main/resources/postgresql.conf new file mode 100644 index 000000000000..dadba56be1b5 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/postgresql.conf @@ -0,0 +1,783 @@ +# ----------------------------- +# PostgreSQL configuration file +# ----------------------------- +# +# This file consists of lines of the form: +# +# name = value +# +# (The "=" is optional.) Whitespace may be used. Comments are introduced with +# "#" anywhere on a line. The complete list of parameter names and allowed +# values can be found in the PostgreSQL documentation. +# +# The commented-out settings shown in this file represent the default values. +# Re-commenting a setting is NOT sufficient to revert it to the default value; +# you need to reload the server. +# +# This file is read on server startup and when the server receives a SIGHUP +# signal. If you edit the file on a running system, you have to SIGHUP the +# server for the changes to take effect, run "pg_ctl reload", or execute +# "SELECT pg_reload_conf()". Some parameters, which are marked below, +# require a server shutdown and restart to take effect. +# +# Any parameter can also be given as a command-line option to the server, e.g., +# "postgres -c log_connections=on". Some parameters can be changed at run time +# with the "SET" SQL command. +# +# Memory units: kB = kilobytes Time units: ms = milliseconds +# MB = megabytes s = seconds +# GB = gigabytes min = minutes +# TB = terabytes h = hours +# d = days + + +#------------------------------------------------------------------------------ +# FILE LOCATIONS +#------------------------------------------------------------------------------ + +# The default values of these variables are driven from the -D command-line +# option or PGDATA environment variable, represented here as ConfigDir. + +#data_directory = 'ConfigDir' # use data in another directory + # (change requires restart) +#hba_file = 'ConfigDir/pg_hba.conf' # host-based authentication file + # (change requires restart) +#ident_file = 'ConfigDir/pg_ident.conf' # ident configuration file + # (change requires restart) + +# If external_pid_file is not explicitly set, no extra PID file is written. +#external_pid_file = '' # write an extra PID file + # (change requires restart) + + +#------------------------------------------------------------------------------ +# CONNECTIONS AND AUTHENTICATION +#------------------------------------------------------------------------------ + +# - Connection Settings - + +listen_addresses = '*' + # comma-separated list of addresses; + # defaults to 'localhost'; use '*' for all + # (change requires restart) +#port = 5432 # (change requires restart) +#max_connections = 100 # (change requires restart) +#superuser_reserved_connections = 3 # (change requires restart) +#unix_socket_directories = '/tmp' # comma-separated list of directories + # (change requires restart) +#unix_socket_group = '' # (change requires restart) +#unix_socket_permissions = 0777 # begin with 0 to use octal notation + # (change requires restart) +#bonjour = off # advertise server via Bonjour + # (change requires restart) +#bonjour_name = '' # defaults to the computer name + # (change requires restart) + +# - TCP settings - +# see "man tcp" for details + +#tcp_keepalives_idle = 0 # TCP_KEEPIDLE, in seconds; + # 0 selects the system default +#tcp_keepalives_interval = 0 # TCP_KEEPINTVL, in seconds; + # 0 selects the system default +#tcp_keepalives_count = 0 # TCP_KEEPCNT; + # 0 selects the system default +#tcp_user_timeout = 0 # TCP_USER_TIMEOUT, in milliseconds; + # 0 selects the system default + +# - Authentication - + +#authentication_timeout = 1min # 1s-600s +#password_encryption = md5 # md5 or scram-sha-256 +#db_user_namespace = off + +# GSSAPI using Kerberos +#krb_server_keyfile = '' +#krb_caseins_users = off + +# - SSL - + +#ssl = off +#ssl_ca_file = '' +#ssl_cert_file = 'server.crt' +#ssl_crl_file = '' +#ssl_key_file = 'server.key' +#ssl_ciphers = 'HIGH:MEDIUM:+3DES:!aNULL' # allowed SSL ciphers +#ssl_prefer_server_ciphers = on +#ssl_ecdh_curve = 'prime256v1' +#ssl_min_protocol_version = 'TLSv1.2' +#ssl_max_protocol_version = '' +#ssl_dh_params_file = '' +#ssl_passphrase_command = '' +#ssl_passphrase_command_supports_reload = off + + +#------------------------------------------------------------------------------ +# RESOURCE USAGE (except WAL) +#------------------------------------------------------------------------------ + +# - Memory - + +#shared_buffers = 32MB # min 128kB + # (change requires restart) +#huge_pages = try # on, off, or try + # (change requires restart) +#temp_buffers = 8MB # min 800kB +#max_prepared_transactions = 0 # zero disables the feature + # (change requires restart) +# Caution: it is not advisable to set max_prepared_transactions nonzero unless +# you actively intend to use prepared transactions. +#work_mem = 4MB # min 64kB +#hash_mem_multiplier = 1.0 # 1-1000.0 multiplier on hash table work_mem +#maintenance_work_mem = 64MB # min 1MB +#autovacuum_work_mem = -1 # min 1MB, or -1 to use maintenance_work_mem +#logical_decoding_work_mem = 64MB # min 64kB +#max_stack_depth = 2MB # min 100kB +#shared_memory_type = mmap # the default is the first option + # supported by the operating system: + # mmap + # sysv + # windows + # (change requires restart) +#dynamic_shared_memory_type = posix # the default is the first option + # supported by the operating system: + # posix + # sysv + # windows + # mmap + # (change requires restart) + +# - Disk - + +#temp_file_limit = -1 # limits per-process temp file space + # in kilobytes, or -1 for no limit + +# - Kernel Resources - + +#max_files_per_process = 1000 # min 64 + # (change requires restart) + +# - Cost-Based Vacuum Delay - + +#vacuum_cost_delay = 0 # 0-100 milliseconds (0 disables) +#vacuum_cost_page_hit = 1 # 0-10000 credits +#vacuum_cost_page_miss = 10 # 0-10000 credits +#vacuum_cost_page_dirty = 20 # 0-10000 credits +#vacuum_cost_limit = 200 # 1-10000 credits + +# - Background Writer - + +#bgwriter_delay = 200ms # 10-10000ms between rounds +#bgwriter_lru_maxpages = 100 # max buffers written/round, 0 disables +#bgwriter_lru_multiplier = 2.0 # 0-10.0 multiplier on buffers scanned/round +#bgwriter_flush_after = 0 # measured in pages, 0 disables + +# - Asynchronous Behavior - + +#effective_io_concurrency = 1 # 1-1000; 0 disables prefetching +#maintenance_io_concurrency = 10 # 1-1000; 0 disables prefetching +#max_worker_processes = 8 # (change requires restart) +#max_parallel_maintenance_workers = 2 # taken from max_parallel_workers +#max_parallel_workers_per_gather = 2 # taken from max_parallel_workers +#parallel_leader_participation = on +#max_parallel_workers = 8 # maximum number of max_worker_processes that + # can be used in parallel operations +#old_snapshot_threshold = -1 # 1min-60d; -1 disables; 0 is immediate + # (change requires restart) +#backend_flush_after = 0 # measured in pages, 0 disables + + +#------------------------------------------------------------------------------ +# WRITE-AHEAD LOG +#------------------------------------------------------------------------------ + +# - Settings - + +#wal_level = replica # minimal, replica, or logical + # (change requires restart) +#fsync = on # flush data to disk for crash safety + # (turning this off can cause + # unrecoverable data corruption) +#synchronous_commit = on # synchronization level; + # off, local, remote_write, remote_apply, or on +#wal_sync_method = fsync # the default is the first option + # supported by the operating system: + # open_datasync + # fdatasync (default on Linux) + # fsync + # fsync_writethrough + # open_sync +#full_page_writes = on # recover from partial page writes +#wal_compression = off # enable compression of full-page writes +#wal_log_hints = off # also do full page writes of non-critical updates + # (change requires restart) +#wal_init_zero = on # zero-fill new WAL files +#wal_recycle = on # recycle WAL files +#wal_buffers = -1 # min 32kB, -1 sets based on shared_buffers + # (change requires restart) +#wal_writer_delay = 200ms # 1-10000 milliseconds +#wal_writer_flush_after = 1MB # measured in pages, 0 disables +#wal_skip_threshold = 2MB + +#commit_delay = 0 # range 0-100000, in microseconds +#commit_siblings = 5 # range 1-1000 + +# - Checkpoints - + +#checkpoint_timeout = 5min # range 30s-1d +#max_wal_size = 1GB +#min_wal_size = 80MB +#checkpoint_completion_target = 0.5 # checkpoint target duration, 0.0 - 1.0 +#checkpoint_flush_after = 0 # measured in pages, 0 disables +#checkpoint_warning = 30s # 0 disables + +# - Archiving - + +#archive_mode = off # enables archiving; off, on, or always + # (change requires restart) +#archive_command = '' # command to use to archive a logfile segment + # placeholders: %p = path of file to archive + # %f = file name only + # e.g. 'test ! -f /mnt/server/archivedir/%f && cp %p /mnt/server/archivedir/%f' +#archive_timeout = 0 # force a logfile segment switch after this + # number of seconds; 0 disables + +# - Archive Recovery - + +# These are only used in recovery mode. + +#restore_command = '' # command to use to restore an archived logfile segment + # placeholders: %p = path of file to restore + # %f = file name only + # e.g. 'cp /mnt/server/archivedir/%f %p' + # (change requires restart) +#archive_cleanup_command = '' # command to execute at every restartpoint +#recovery_end_command = '' # command to execute at completion of recovery + +# - Recovery Target - + +# Set these only when performing a targeted recovery. + +#recovery_target = '' # 'immediate' to end recovery as soon as a + # consistent state is reached + # (change requires restart) +#recovery_target_name = '' # the named restore point to which recovery will proceed + # (change requires restart) +#recovery_target_time = '' # the time stamp up to which recovery will proceed + # (change requires restart) +#recovery_target_xid = '' # the transaction ID up to which recovery will proceed + # (change requires restart) +#recovery_target_lsn = '' # the WAL LSN up to which recovery will proceed + # (change requires restart) +#recovery_target_inclusive = on # Specifies whether to stop: + # just after the specified recovery target (on) + # just before the recovery target (off) + # (change requires restart) +#recovery_target_timeline = 'latest' # 'current', 'latest', or timeline ID + # (change requires restart) +#recovery_target_action = 'pause' # 'pause', 'promote', 'shutdown' + # (change requires restart) + + +#------------------------------------------------------------------------------ +# REPLICATION +#------------------------------------------------------------------------------ + +# - Sending Servers - + +# Set these on the master and on any standby that will send replication data. + +#max_wal_senders = 10 # max number of walsender processes + # (change requires restart) +#wal_keep_size = 0 # in megabytes; 0 disables +#max_slot_wal_keep_size = -1 # in megabytes; -1 disables +#wal_sender_timeout = 60s # in milliseconds; 0 disables + +#max_replication_slots = 10 # max number of replication slots + # (change requires restart) +#track_commit_timestamp = off # collect timestamp of transaction commit + # (change requires restart) + +# - Master Server - + +# These settings are ignored on a standby server. + +#synchronous_standby_names = '' # standby servers that provide sync rep + # method to choose sync standbys, number of sync standbys, + # and comma-separated list of application_name + # from standby(s); '*' = all +#vacuum_defer_cleanup_age = 0 # number of xacts by which cleanup is delayed + +# - Standby Servers - + +# These settings are ignored on a master server. + +#primary_conninfo = '' # connection string to sending server +#primary_slot_name = '' # replication slot on sending server +#promote_trigger_file = '' # file name whose presence ends recovery +#hot_standby = on # "off" disallows queries during recovery + # (change requires restart) +#max_standby_archive_delay = 30s # max delay before canceling queries + # when reading WAL from archive; + # -1 allows indefinite delay +#max_standby_streaming_delay = 30s # max delay before canceling queries + # when reading streaming WAL; + # -1 allows indefinite delay +#wal_receiver_create_temp_slot = off # create temp slot if primary_slot_name + # is not set +#wal_receiver_status_interval = 10s # send replies at least this often + # 0 disables +#hot_standby_feedback = off # send info from standby to prevent + # query conflicts +#wal_receiver_timeout = 60s # time that receiver waits for + # communication from master + # in milliseconds; 0 disables +#wal_retrieve_retry_interval = 5s # time to wait before retrying to + # retrieve WAL after a failed attempt +#recovery_min_apply_delay = 0 # minimum delay for applying changes during recovery + +# - Subscribers - + +# These settings are ignored on a publisher. + +#max_logical_replication_workers = 4 # taken from max_worker_processes + # (change requires restart) +#max_sync_workers_per_subscription = 2 # taken from max_logical_replication_workers + + +#------------------------------------------------------------------------------ +# QUERY TUNING +#------------------------------------------------------------------------------ + +# - Planner Method Configuration - + +#enable_bitmapscan = on +#enable_hashagg = on +#enable_hashjoin = on +#enable_indexscan = on +#enable_indexonlyscan = on +#enable_material = on +#enable_mergejoin = on +#enable_nestloop = on +#enable_parallel_append = on +#enable_seqscan = on +#enable_sort = on +#enable_incremental_sort = on +#enable_tidscan = on +#enable_partitionwise_join = off +#enable_partitionwise_aggregate = off +#enable_parallel_hash = on +#enable_partition_pruning = on + +# - Planner Cost Constants - + +#seq_page_cost = 1.0 # measured on an arbitrary scale +#random_page_cost = 4.0 # same scale as above +#cpu_tuple_cost = 0.01 # same scale as above +#cpu_index_tuple_cost = 0.005 # same scale as above +#cpu_operator_cost = 0.0025 # same scale as above +#parallel_tuple_cost = 0.1 # same scale as above +#parallel_setup_cost = 1000.0 # same scale as above + +#jit_above_cost = 100000 # perform JIT compilation if available + # and query more expensive than this; + # -1 disables +#jit_inline_above_cost = 500000 # inline small functions if query is + # more expensive than this; -1 disables +#jit_optimize_above_cost = 500000 # use expensive JIT optimizations if + # query is more expensive than this; + # -1 disables + +#min_parallel_table_scan_size = 8MB +#min_parallel_index_scan_size = 512kB +#effective_cache_size = 4GB + +# - Genetic Query Optimizer - + +#geqo = on +#geqo_threshold = 12 +#geqo_effort = 5 # range 1-10 +#geqo_pool_size = 0 # selects default based on effort +#geqo_generations = 0 # selects default based on effort +#geqo_selection_bias = 2.0 # range 1.5-2.0 +#geqo_seed = 0.0 # range 0.0-1.0 + +# - Other Planner Options - + +#default_statistics_target = 100 # range 1-10000 +#constraint_exclusion = partition # on, off, or partition +#cursor_tuple_fraction = 0.1 # range 0.0-1.0 +#from_collapse_limit = 8 +#join_collapse_limit = 8 # 1 disables collapsing of explicit + # JOIN clauses +#force_parallel_mode = off +#jit = on # allow JIT compilation +#plan_cache_mode = auto # auto, force_generic_plan or + # force_custom_plan + + +#------------------------------------------------------------------------------ +# REPORTING AND LOGGING +#------------------------------------------------------------------------------ + +# - Where to Log - + +#log_destination = 'stderr' # Valid values are combinations of + # stderr, csvlog, syslog, and eventlog, + # depending on platform. csvlog + # requires logging_collector to be on. + +# This is used when logging to stderr: +#logging_collector = off # Enable capturing of stderr and csvlog + # into log files. Required to be on for + # csvlogs. + # (change requires restart) + +# These are only used if logging_collector is on: +#log_directory = 'log' # directory where log files are written, + # can be absolute or relative to PGDATA +#log_filename = 'postgresql-%Y-%m-%d_%H%M%S.log' # log file name pattern, + # can include strftime() escapes +#log_file_mode = 0600 # creation mode for log files, + # begin with 0 to use octal notation +#log_truncate_on_rotation = off # If on, an existing log file with the + # same name as the new log file will be + # truncated rather than appended to. + # But such truncation only occurs on + # time-driven rotation, not on restarts + # or size-driven rotation. Default is + # off, meaning append to existing files + # in all cases. +#log_rotation_age = 1d # Automatic rotation of logfiles will + # happen after that time. 0 disables. +#log_rotation_size = 10MB # Automatic rotation of logfiles will + # happen after that much log output. + # 0 disables. + +# These are relevant when logging to syslog: +#syslog_facility = 'LOCAL0' +#syslog_ident = 'postgres' +#syslog_sequence_numbers = on +#syslog_split_messages = on + +# This is only relevant when logging to eventlog (win32): +# (change requires restart) +#event_source = 'PostgreSQL' + +# - When to Log - + +#log_min_messages = warning # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # info + # notice + # warning + # error + # log + # fatal + # panic + +#log_min_error_statement = error # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # info + # notice + # warning + # error + # log + # fatal + # panic (effectively off) + +#log_min_duration_statement = -1 # -1 is disabled, 0 logs all statements + # and their durations, > 0 logs only + # statements running at least this number + # of milliseconds + +#log_min_duration_sample = -1 # -1 is disabled, 0 logs a sample of statements + # and their durations, > 0 logs only a sample of + # statements running at least this number + # of milliseconds; + # sample fraction is determined by log_statement_sample_rate + +#log_statement_sample_rate = 1.0 # fraction of logged statements exceeding + # log_min_duration_sample to be logged; + # 1.0 logs all such statements, 0.0 never logs + + +#log_transaction_sample_rate = 0.0 # fraction of transactions whose statements + # are logged regardless of their duration; 1.0 logs all + # statements from all transactions, 0.0 never logs + +# - What to Log - + +#debug_print_parse = off +#debug_print_rewritten = off +#debug_print_plan = off +#debug_pretty_print = on +#log_checkpoints = off +#log_connections = off +#log_disconnections = off +#log_duration = off +#log_error_verbosity = default # terse, default, or verbose messages +#log_hostname = off +#log_line_prefix = '%m [%p] ' # special values: + # %a = application name + # %u = user name + # %d = database name + # %r = remote host and port + # %h = remote host + # %b = backend type + # %p = process ID + # %t = timestamp without milliseconds + # %m = timestamp with milliseconds + # %n = timestamp with milliseconds (as a Unix epoch) + # %i = command tag + # %e = SQL state + # %c = session ID + # %l = session line number + # %s = session start timestamp + # %v = virtual transaction ID + # %x = transaction ID (0 if none) + # %q = stop here in non-session + # processes + # %% = '%' + # e.g. '<%u%%%d> ' +#log_lock_waits = off # log lock waits >= deadlock_timeout +#log_parameter_max_length = -1 # when logging statements, limit logged + # bind-parameter values to N bytes; + # -1 means print in full, 0 disables +#log_parameter_max_length_on_error = 0 # when logging an error, limit logged + # bind-parameter values to N bytes; + # -1 means print in full, 0 disables +#log_statement = 'none' # none, ddl, mod, all +#log_replication_commands = off +#log_temp_files = -1 # log temporary files equal or larger + # than the specified size in kilobytes; + # -1 disables, 0 logs all temp files +#log_timezone = 'GMT' + +#------------------------------------------------------------------------------ +# PROCESS TITLE +#------------------------------------------------------------------------------ + +#cluster_name = '' # added to process titles if nonempty + # (change requires restart) +#update_process_title = on + + +#------------------------------------------------------------------------------ +# STATISTICS +#------------------------------------------------------------------------------ + +# - Query and Index Statistics Collector - + +#track_activities = on +#track_counts = on +#track_io_timing = off +#track_functions = none # none, pl, all +#track_activity_query_size = 1024 # (change requires restart) +#stats_temp_directory = 'pg_stat_tmp' + + +# - Monitoring - + +#log_parser_stats = off +#log_planner_stats = off +#log_executor_stats = off +#log_statement_stats = off + + +#------------------------------------------------------------------------------ +# AUTOVACUUM +#------------------------------------------------------------------------------ + +#autovacuum = on # Enable autovacuum subprocess? 'on' + # requires track_counts to also be on. +#log_autovacuum_min_duration = -1 # -1 disables, 0 logs all actions and + # their durations, > 0 logs only + # actions running at least this number + # of milliseconds. +#autovacuum_max_workers = 3 # max number of autovacuum subprocesses + # (change requires restart) +#autovacuum_naptime = 1min # time between autovacuum runs +#autovacuum_vacuum_threshold = 50 # min number of row updates before + # vacuum +#autovacuum_vacuum_insert_threshold = 1000 # min number of row inserts + # before vacuum; -1 disables insert + # vacuums +#autovacuum_analyze_threshold = 50 # min number of row updates before + # analyze +#autovacuum_vacuum_scale_factor = 0.2 # fraction of table size before vacuum +#autovacuum_vacuum_insert_scale_factor = 0.2 # fraction of inserts over table + # size before insert vacuum +#autovacuum_analyze_scale_factor = 0.1 # fraction of table size before analyze +#autovacuum_freeze_max_age = 200000000 # maximum XID age before forced vacuum + # (change requires restart) +#autovacuum_multixact_freeze_max_age = 400000000 # maximum multixact age + # before forced vacuum + # (change requires restart) +#autovacuum_vacuum_cost_delay = 2ms # default vacuum cost delay for + # autovacuum, in milliseconds; + # -1 means use vacuum_cost_delay +#autovacuum_vacuum_cost_limit = -1 # default vacuum cost limit for + # autovacuum, -1 means use + # vacuum_cost_limit + + +#------------------------------------------------------------------------------ +# CLIENT CONNECTION DEFAULTS +#------------------------------------------------------------------------------ + +# - Statement Behavior - + +#client_min_messages = notice # values in order of decreasing detail: + # debug5 + # debug4 + # debug3 + # debug2 + # debug1 + # log + # notice + # warning + # error +#search_path = '"$user", public' # schema names +#row_security = on +#default_tablespace = '' # a tablespace name, '' uses the default +#temp_tablespaces = '' # a list of tablespace names, '' uses + # only default tablespace +#default_table_access_method = 'heap' +#check_function_bodies = on +#default_transaction_isolation = 'read committed' +#default_transaction_read_only = off +#default_transaction_deferrable = off +#session_replication_role = 'origin' +#statement_timeout = 0 # in milliseconds, 0 is disabled +#lock_timeout = 0 # in milliseconds, 0 is disabled +#idle_in_transaction_session_timeout = 0 # in milliseconds, 0 is disabled +#vacuum_freeze_min_age = 50000000 +#vacuum_freeze_table_age = 150000000 +#vacuum_multixact_freeze_min_age = 5000000 +#vacuum_multixact_freeze_table_age = 150000000 +#vacuum_cleanup_index_scale_factor = 0.1 # fraction of total number of tuples + # before index cleanup, 0 always performs + # index cleanup +#bytea_output = 'hex' # hex, escape +#xmlbinary = 'base64' +#xmloption = 'content' +#gin_fuzzy_search_limit = 0 +#gin_pending_list_limit = 4MB + +# - Locale and Formatting - + +#datestyle = 'iso, mdy' +#intervalstyle = 'postgres' +#timezone = 'GMT' +#timezone_abbreviations = 'Default' # Select the set of available time zone + # abbreviations. Currently, there are + # Default + # Australia (historical usage) + # India + # You can create your own file in + # share/timezonesets/. +#extra_float_digits = 1 # min -15, max 3; any value >0 actually + # selects precise output mode +#client_encoding = sql_ascii # actually, defaults to database + # encoding + +# These settings are initialized by initdb, but they can be changed. +#lc_messages = 'C' # locale for system error message + # strings +#lc_monetary = 'C' # locale for monetary formatting +#lc_numeric = 'C' # locale for number formatting +#lc_time = 'C' # locale for time formatting + +# default configuration for text search +#default_text_search_config = 'pg_catalog.simple' + +# - Shared Library Preloading - + +#shared_preload_libraries = '' # (change requires restart) +#local_preload_libraries = '' +#session_preload_libraries = '' +#jit_provider = 'llvmjit' # JIT library to use + +# - Other Defaults - + +#dynamic_library_path = '$libdir' +#extension_destdir = '' # prepend path when loading extensions + # and shared objects (added by Debian) + + +#------------------------------------------------------------------------------ +# LOCK MANAGEMENT +#------------------------------------------------------------------------------ + +#deadlock_timeout = 1s +#max_locks_per_transaction = 64 # min 10 + # (change requires restart) +#max_pred_locks_per_transaction = 64 # min 10 + # (change requires restart) +#max_pred_locks_per_relation = -2 # negative values mean + # (max_pred_locks_per_transaction + # / -max_pred_locks_per_relation) - 1 +#max_pred_locks_per_page = 2 # min 0 + + +#------------------------------------------------------------------------------ +# VERSION AND PLATFORM COMPATIBILITY +#------------------------------------------------------------------------------ + +# - Previous PostgreSQL Versions - + +#array_nulls = on +#backslash_quote = safe_encoding # on, off, or safe_encoding +#escape_string_warning = on +#lo_compat_privileges = off +#operator_precedence_warning = off +#quote_all_identifiers = off +#standard_conforming_strings = on +#synchronize_seqscans = on + +# - Other Platforms and Clients - + +#transform_null_equals = off + + +#------------------------------------------------------------------------------ +# ERROR HANDLING +#------------------------------------------------------------------------------ + +#exit_on_error = off # terminate session on any error? +#restart_after_crash = on # reinitialize after backend crash? +#data_sync_retry = off # retry or panic on failure to fsync + # data? + # (change requires restart) + + +#------------------------------------------------------------------------------ +# CONFIG FILE INCLUDES +#------------------------------------------------------------------------------ + +# These options allow settings to be loaded from files other than the +# default postgresql.conf. Note that these are directives, not variable +# assignments, so they can usefully be given more than once. + +#include_dir = '...' # include files ending in '.conf' from + # a directory, e.g., 'conf.d' +#include_if_exists = '...' # include file only if it exists +#include = '...' # include file + + +#------------------------------------------------------------------------------ +# CUSTOMIZED OPTIONS +#------------------------------------------------------------------------------ +wal_level = logical +max_wal_senders = 10 +max_replication_slots = 10 + diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/spec.json b/airbyte-integrations/connectors/source-postgres/src/main/resources/spec.json index ebe9c1658abb..4361ac74d95f 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/spec.json +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/spec.json @@ -10,7 +10,8 @@ "host": { "title": "Host", "description": "Hostname of the database.", - "type": "string" + "type": "string", + "order": 0 }, "port": { "title": "Port", @@ -19,23 +20,58 @@ "minimum": 0, "maximum": 65536, "default": 5432, - "examples": ["5432"] + "examples": ["5432"], + "order": 1 }, "database": { "title": "DB Name", "description": "Name of the database.", - "type": "string" + "type": "string", + "order": 2 }, "username": { "title": "User", "description": "Username to use to access the database.", - "type": "string" + "type": "string", + "order": 3 }, "password": { "title": "Password", "description": "Password associated with the username.", "type": "string", - "airbyte_secret": true + "airbyte_secret": true, + "order": 4 + }, + "replication_method": { + "type": "object", + "title": "Replication Method", + "description": "Replication method to use for extracting data from the database.", + "order": 5, + "oneOf": [ + { + "title": "Standard", + "additionalProperties": false, + "description": "Standard replication requires no setup on the DB side but will not be able to represent deletions incrementally.", + "required": [], + "properties": {} + }, + { + "title": "Logical Replication (CDC)", + "additionalProperties": false, + "description": "Logical replication uses the Postgres write-ahead log (WAL) to detect inserts, updates, and deletes. This needs to be configured on the source database itself. Only available on Postgres 10 and above. Read the Postgres Source docs for more information.", + "required": ["replication_slot", "publication"], + "properties": { + "replication_slot": { + "type": "string", + "description": "A pgoutput logical replication slot." + }, + "publication": { + "type": "string", + "description": "A Postgres publication used for consuming changes." + } + } + } + ] } } } diff --git a/airbyte-integrations/connectors/source-postgres/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcPostgresSourceStandardTest.java b/airbyte-integrations/connectors/source-postgres/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcPostgresSourceStandardTest.java new file mode 100644 index 000000000000..b757effd354e --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test-integration/java/io/airbyte/integrations/io/airbyte/integration_tests/sources/CdcPostgresSourceStandardTest.java @@ -0,0 +1,153 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.io.airbyte.integration_tests.sources; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.db.Database; +import io.airbyte.db.Databases; +import io.airbyte.integrations.standardtest.source.StandardSourceTest; +import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.ConnectorSpecification; +import io.airbyte.protocol.models.DestinationSyncMode; +import io.airbyte.protocol.models.Field; +import io.airbyte.protocol.models.Field.JsonSchemaPrimitive; +import io.airbyte.protocol.models.SyncMode; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import org.jooq.SQLDialect; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.utility.MountableFile; + +// todo (cgardens) - Sanity check that when configured for CDC that postgres performs like any other +// incremental source. As we have more sources support CDC we will find a more reusable way of doing +// this, but for now this is a solid sanity check. +public class CdcPostgresSourceStandardTest extends StandardSourceTest { + + private static final String SLOT_NAME_BASE = "debezium_slot"; + private static final String STREAM_NAME = "public.id_and_name"; + private static final String STREAM_NAME2 = "public.starships"; + + private PostgreSQLContainer container; + private JsonNode config; + + @Override + protected void setup(TestDestinationEnv testEnv) throws Exception { + container = new PostgreSQLContainer<>("postgres:13-alpine") + .withCopyFileToContainer(MountableFile.forClasspathResource("postgresql.conf"), "/etc/postgresql/postgresql.conf") + .withCommand("postgres -c config_file=/etc/postgresql/postgresql.conf"); + container.start(); + + config = Jsons.jsonNode(ImmutableMap.builder() + .put("host", container.getHost()) + .put("port", container.getFirstMappedPort()) + .put("database", container.getDatabaseName()) + .put("username", container.getUsername()) + .put("password", container.getPassword()) + .put("replication_method", ImmutableMap.of("replication_slot", SLOT_NAME_BASE)) + .build()); + + final Database database = Databases.createDatabase( + config.get("username").asText(), + config.get("password").asText(), + String.format("jdbc:postgresql://%s:%s/%s", + config.get("host").asText(), + config.get("port").asText(), + config.get("database").asText()), + "org.postgresql.Driver", + SQLDialect.POSTGRES); + + database.query(ctx -> { + ctx.execute("SELECT pg_create_logical_replication_slot('" + SLOT_NAME_BASE + "', 'pgoutput');"); + ctx.execute("CREATE TABLE id_and_name(id INTEGER, name VARCHAR(200));"); + ctx.execute("INSERT INTO id_and_name (id, name) VALUES (1,'picard'), (2, 'crusher'), (3, 'vash');"); + ctx.execute("CREATE TABLE starships(id INTEGER, name VARCHAR(200));"); + ctx.execute("INSERT INTO starships (id, name) VALUES (1,'enterprise-d'), (2, 'defiant'), (3, 'yamato');"); + return null; + }); + + database.close(); + } + + @Override + protected void tearDown(TestDestinationEnv testEnv) { + container.close(); + } + + @Override + protected String getImageName() { + return "airbyte/source-postgres:dev"; + } + + @Override + protected ConnectorSpecification getSpec() throws Exception { + return Jsons.deserialize(MoreResources.readResource("spec.json"), ConnectorSpecification.class); + } + + @Override + protected JsonNode getConfig() { + return config; + } + + @Override + protected ConfiguredAirbyteCatalog getConfiguredCatalog() { + return new ConfiguredAirbyteCatalog().withStreams(Lists.newArrayList( + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(Lists.newArrayList("id")) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withStream(CatalogHelpers.createAirbyteStream( + STREAM_NAME, + Field.of("id", JsonSchemaPrimitive.NUMBER), + Field.of("name", JsonSchemaPrimitive.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))), + new ConfiguredAirbyteStream() + .withSyncMode(SyncMode.INCREMENTAL) + .withCursorField(Lists.newArrayList("id")) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withStream(CatalogHelpers.createAirbyteStream( + STREAM_NAME2, + Field.of("id", JsonSchemaPrimitive.NUMBER), + Field.of("name", JsonSchemaPrimitive.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL))))); + } + + @Override + protected List getRegexTests() { + return Collections.emptyList(); + } + + @Override + protected JsonNode getState() { + return Jsons.jsonNode(new HashMap<>()); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStoreTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStoreTest.java new file mode 100644 index 000000000000..f5638fec8914 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/AirbyteFileOffsetBackingStoreTest.java @@ -0,0 +1,65 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import io.airbyte.commons.io.IOs; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.integrations.source.jdbc.models.CdcState; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.junit.jupiter.api.Test; + +class AirbyteFileOffsetBackingStoreTest { + + @SuppressWarnings("UnstableApiUsage") + @Test + void test() throws IOException { + final Path testRoot = Files.createTempDirectory(Path.of("/tmp"), "offset-store-test"); + + final byte[] bytes = MoreResources.readBytes("test_debezium_offset.dat"); + final Path templateFilePath = testRoot.resolve("template_offset.dat"); + IOs.writeFile(templateFilePath, bytes); + + final Path writeFilePath = testRoot.resolve("offset.dat"); + + final AirbyteFileOffsetBackingStore offsetStore = new AirbyteFileOffsetBackingStore(templateFilePath); + final CdcState stateFromTemplateFile = offsetStore.read(); + + final AirbyteFileOffsetBackingStore offsetStore2 = new AirbyteFileOffsetBackingStore(writeFilePath); + offsetStore2.persist(stateFromTemplateFile); + + final CdcState stateFromOffsetStoreRoundTrip = offsetStore2.read(); + + // verify that, after a round trip through the offset store, we get back the same data. + assertEquals(stateFromTemplateFile, stateFromOffsetStoreRoundTrip); + // verify that the file written by the offset store is identical to the template file. + assertTrue(com.google.common.io.Files.equal(templateFilePath.toFile(), writeFilePath.toFile())); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/CdcPostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/CdcPostgresSourceTest.java new file mode 100644 index 000000000000..6ba0c6d54a32 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/CdcPostgresSourceTest.java @@ -0,0 +1,544 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import static java.lang.Thread.sleep; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import com.google.common.collect.Streams; +import io.airbyte.commons.io.IOs; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.lang.Exceptions; +import io.airbyte.commons.util.AutoCloseableIterator; +import io.airbyte.commons.util.AutoCloseableIterators; +import io.airbyte.db.Database; +import io.airbyte.db.Databases; +import io.airbyte.integrations.source.jdbc.AbstractJdbcSource; +import io.airbyte.protocol.models.AirbyteCatalog; +import io.airbyte.protocol.models.AirbyteConnectionStatus; +import io.airbyte.protocol.models.AirbyteMessage; +import io.airbyte.protocol.models.AirbyteMessage.Type; +import io.airbyte.protocol.models.AirbyteRecordMessage; +import io.airbyte.protocol.models.AirbyteStateMessage; +import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.Field; +import io.airbyte.protocol.models.Field.JsonSchemaPrimitive; +import io.airbyte.protocol.models.SyncMode; +import io.airbyte.test.utils.PostgreSQLContainerHelper; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.stream.Collectors; +import java.util.stream.Stream; +import org.apache.commons.lang3.RandomStringUtils; +import org.jooq.DSLContext; +import org.jooq.SQLDialect; +import org.junit.jupiter.api.AfterAll; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.DisplayName; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; +import org.testcontainers.containers.PostgreSQLContainer; +import org.testcontainers.utility.MountableFile; + +class CdcPostgresSourceTest { + + private static final Logger LOGGER = LoggerFactory.getLogger(CdcPostgresSourceTest.class); + + private static final String SLOT_NAME_BASE = "debezium_slot"; + private static final String MAKES_STREAM_NAME = "public.makes"; + private static final String MODELS_STREAM_NAME = "public.models"; + private static final Set STREAM_NAMES = Sets.newHashSet(MAKES_STREAM_NAME, MODELS_STREAM_NAME); + private static final String COL_ID = "id"; + private static final String COL_MAKE = "make"; + private static final String COL_MAKE_ID = "make_id"; + private static final String COL_MODEL = "model"; + private static final String PUBLICATION = "publication"; + + private static final AirbyteCatalog CATALOG = new AirbyteCatalog().withStreams(List.of( + CatalogHelpers.createAirbyteStream( + MAKES_STREAM_NAME, + Field.of(COL_ID, JsonSchemaPrimitive.NUMBER), + Field.of(COL_MAKE, JsonSchemaPrimitive.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) + .withSourceDefinedPrimaryKey(List.of(List.of(COL_ID))), + CatalogHelpers.createAirbyteStream( + MODELS_STREAM_NAME, + Field.of(COL_ID, JsonSchemaPrimitive.NUMBER), + Field.of(COL_MAKE_ID, JsonSchemaPrimitive.NUMBER), + Field.of(COL_MODEL, JsonSchemaPrimitive.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) + .withSourceDefinedPrimaryKey(List.of(List.of(COL_ID))))); + private static final ConfiguredAirbyteCatalog CONFIGURED_CATALOG = CatalogHelpers.toDefaultConfiguredCatalog(CATALOG); + + // set all streams to incremental. + static { + CONFIGURED_CATALOG.getStreams().forEach(s -> s.setSyncMode(SyncMode.INCREMENTAL)); + } + + private static final List MAKE_RECORDS = ImmutableList.of( + Jsons.jsonNode(ImmutableMap.of(COL_ID, 1, COL_MAKE, "Ford")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 2, COL_MAKE, "Mercedes"))); + + private static final List MODEL_RECORDS = ImmutableList.of( + Jsons.jsonNode(ImmutableMap.of(COL_ID, 11, COL_MAKE_ID, 1, COL_MODEL, "Fiesta")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 12, COL_MAKE_ID, 1, COL_MODEL, "Focus")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 13, COL_MAKE_ID, 1, COL_MODEL, "Ranger")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 14, COL_MAKE_ID, 2, COL_MODEL, "GLA")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 15, COL_MAKE_ID, 2, COL_MODEL, "A 220")), + Jsons.jsonNode(ImmutableMap.of(COL_ID, 16, COL_MAKE_ID, 2, COL_MODEL, "E 350"))); + + private static PostgreSQLContainer PSQL_DB; + + private String dbName; + private Database database; + private PostgresSource source; + + @BeforeAll + static void init() { + PSQL_DB = new PostgreSQLContainer<>("postgres:13-alpine") + .withCopyFileToContainer(MountableFile.forClasspathResource("postgresql.conf"), "/etc/postgresql/postgresql.conf") + .withCommand("postgres -c config_file=/etc/postgresql/postgresql.conf"); + PSQL_DB.start(); + } + + @AfterAll + static void tearDown() { + PSQL_DB.close(); + } + + @BeforeEach + void setup() throws Exception { + source = new PostgresSource(); + + dbName = "db_" + RandomStringUtils.randomAlphabetic(10).toLowerCase(); + + final String initScriptName = "init_" + dbName.concat(".sql"); + final String tmpFilePath = IOs.writeFileToRandomTmpDir(initScriptName, "CREATE DATABASE " + dbName + ";"); + PostgreSQLContainerHelper.runSqlScript(MountableFile.forHostPath(tmpFilePath), PSQL_DB); + + final JsonNode config = getConfig(PSQL_DB, dbName); + final String fullReplicationSlot = SLOT_NAME_BASE + "_" + dbName; + database = getDatabaseFromConfig(config); + database.query(ctx -> { + ctx.execute("SELECT pg_create_logical_replication_slot('" + fullReplicationSlot + "', 'pgoutput');"); + ctx.execute("CREATE PUBLICATION " + PUBLICATION + " FOR ALL TABLES;"); + ctx.execute(String.format("CREATE TABLE %s(%s INTEGER, %s VARCHAR(200), PRIMARY KEY (%s));", MAKES_STREAM_NAME, COL_ID, COL_MAKE, COL_ID)); + ctx.execute(String.format("CREATE TABLE %s(%s INTEGER, %s INTEGER, %s VARCHAR(200), PRIMARY KEY (%s));", + MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL, COL_ID)); + + for (JsonNode recordJson : MAKE_RECORDS) { + writeMakeRecord(ctx, recordJson); + } + + for (JsonNode recordJson : MODEL_RECORDS) { + writeModelRecord(ctx, recordJson); + } + + return null; + }); + } + + private JsonNode getConfig(PostgreSQLContainer psqlDb, String dbName) { + final JsonNode replicationMethod = Jsons.jsonNode(ImmutableMap.builder() + .put("replication_slot", SLOT_NAME_BASE + "_" + dbName) + .put("publication", PUBLICATION) + .build()); + + return Jsons.jsonNode(ImmutableMap.builder() + .put("host", psqlDb.getHost()) + .put("port", psqlDb.getFirstMappedPort()) + .put("database", dbName) + .put("username", psqlDb.getUsername()) + .put("password", psqlDb.getPassword()) + .put("replication_method", replicationMethod) + .build()); + } + + private Database getDatabaseFromConfig(JsonNode config) { + return Databases.createDatabase( + config.get("username").asText(), + config.get("password").asText(), + String.format("jdbc:postgresql://%s:%s/%s", + config.get("host").asText(), + config.get("port").asText(), + config.get("database").asText()), + "org.postgresql.Driver", + SQLDialect.POSTGRES); + } + + @Test + @DisplayName("On the first First sync, produces returns records that exist in the database.") + void testExistingData() throws Exception { + final AutoCloseableIterator read = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords = AutoCloseableIterators.toListAndClose(read); + + final Set recordMessages = extractRecordMessages(actualRecords); + final List stateMessages = extractStateMessages(actualRecords); + + assertExpectedRecords(Stream.concat(MAKE_RECORDS.stream(), MODEL_RECORDS.stream()).collect(Collectors.toSet()), recordMessages); + assertExpectedStateMessages(stateMessages); + } + + @Test + @DisplayName("When a record is deleted, produces a deletion record.") + void testDelete() throws Exception { + final AutoCloseableIterator read1 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords1 = AutoCloseableIterators.toListAndClose(read1); + final List stateMessages1 = extractStateMessages(actualRecords1); + + assertExpectedStateMessages(stateMessages1); + + database.query(ctx -> { + ctx.execute(String.format("DELETE FROM %s WHERE %s = %s", MODELS_STREAM_NAME, COL_ID, 11)); + return null; + }); + + final JsonNode state = stateMessages1.get(0).getData(); + final AutoCloseableIterator read2 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, state); + final List actualRecords2 = AutoCloseableIterators.toListAndClose(read2); + final List recordMessages2 = new ArrayList<>(extractRecordMessages(actualRecords2)); + final List stateMessages2 = extractStateMessages(actualRecords2); + + assertExpectedStateMessages(stateMessages2); + assertEquals(1, recordMessages2.size()); + assertEquals(11, recordMessages2.get(0).getData().get(COL_ID).asInt()); + assertNotNull(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_LSN)); + assertNotNull(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_UPDATED_AT)); + assertNotNull(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_DELETED_AT)); + } + + @Test + @DisplayName("When a record is updated, produces an update record.") + void testUpdate() throws Exception { + final String updatedModel = "Explorer"; + final AutoCloseableIterator read1 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords1 = AutoCloseableIterators.toListAndClose(read1); + final List stateMessages1 = extractStateMessages(actualRecords1); + + assertExpectedStateMessages(stateMessages1); + + database.query(ctx -> { + ctx.execute(String.format("UPDATE %s SET %s = '%s' WHERE %s = %s", MODELS_STREAM_NAME, COL_MODEL, updatedModel, COL_ID, 11)); + return null; + }); + + final JsonNode state = stateMessages1.get(0).getData(); + final AutoCloseableIterator read2 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, state); + final List actualRecords2 = AutoCloseableIterators.toListAndClose(read2); + final List recordMessages2 = new ArrayList<>(extractRecordMessages(actualRecords2)); + final List stateMessages2 = extractStateMessages(actualRecords2); + + assertExpectedStateMessages(stateMessages2); + assertEquals(1, recordMessages2.size()); + assertEquals(11, recordMessages2.get(0).getData().get(COL_ID).asInt()); + assertEquals(updatedModel, recordMessages2.get(0).getData().get(COL_MODEL).asText()); + assertNotNull(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_LSN)); + assertNotNull(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_UPDATED_AT)); + assertTrue(recordMessages2.get(0).getData().get(AbstractJdbcSource.CDC_DELETED_AT).isNull()); + } + + @SuppressWarnings({"BusyWait", "CodeBlock2Expr"}) + @Test + @DisplayName("Verify that when data is inserted into the database while a sync is happening and after the first sync, it all gets replicated.") + void testRecordsProducedDuringAndAfterSync() throws Exception { + final int recordsToCreate = 20; + final AtomicInteger recordsCreated = new AtomicInteger(); + final ScheduledExecutorService executorService = Executors.newScheduledThreadPool(1); + executorService.scheduleAtFixedRate(() -> { + Exceptions.toRuntime(() -> database.query(ctx -> { + if (recordsCreated.get() < recordsToCreate) { + final JsonNode record = + Jsons.jsonNode(ImmutableMap.of(COL_ID, 100 + recordsCreated.get(), COL_MAKE_ID, 1, COL_MODEL, "F-" + recordsCreated.get())); + writeModelRecord(ctx, record); + + recordsCreated.incrementAndGet(); + } + return null; + })); + }, 0, 500, TimeUnit.MILLISECONDS); + + final AutoCloseableIterator read1 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords1 = AutoCloseableIterators.toListAndClose(read1); + assertExpectedStateMessages(extractStateMessages(actualRecords1)); + + while (recordsCreated.get() != recordsToCreate) { + LOGGER.info("waiting for records to be created."); + sleep(500); + } + executorService.shutdown(); + + final JsonNode state = extractStateMessages(actualRecords1).get(0).getData(); + final AutoCloseableIterator read2 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, state); + final List actualRecords2 = AutoCloseableIterators.toListAndClose(read2); + + assertExpectedStateMessages(extractStateMessages(actualRecords2)); + + // sometimes there can be more than one of these at the end of the snapshot and just before the + // first incremental. + final Set recordMessages1 = removeDuplicates(extractRecordMessages(actualRecords1)); + final Set recordMessages2 = removeDuplicates(extractRecordMessages(actualRecords2)); + + final int recordsCreatedBeforeTestCount = MAKE_RECORDS.size() + MODEL_RECORDS.size(); + assertTrue(recordsCreatedBeforeTestCount < recordMessages1.size(), "Expected first sync to include records created while the test was running."); + assertTrue(0 < recordMessages2.size(), "Expected records to be replicated in the second sync."); + LOGGER.info("recordsToCreate = " + recordsToCreate); + LOGGER.info("recordsCreatedBeforeTestCount = " + recordsCreatedBeforeTestCount); + LOGGER.info("recordMessages1.size() = " + recordMessages1.size()); + LOGGER.info("recordMessages2.size() = " + recordMessages2.size()); + assertEquals(recordsToCreate + recordsCreatedBeforeTestCount, recordMessages1.size() + recordMessages2.size()); + } + + private static Set removeDuplicates(Set messages) { + final Set existingDataRecordsWithoutUpdated = new HashSet<>(); + final Set output = new HashSet<>(); + + for (AirbyteRecordMessage message : messages) { + ObjectNode node = message.getData().deepCopy(); + node.remove("_ab_cdc_updated_at"); + + if (existingDataRecordsWithoutUpdated.contains(node)) { + LOGGER.info("Removing duplicate node: " + node); + } else { + output.add(message); + existingDataRecordsWithoutUpdated.add(node); + } + } + + return output; + } + + @Test + @DisplayName("When both incremental CDC and full refresh are configured for different streams in a sync, the data is replicated as expected.") + void testCdcAndFullRefreshInSameSync() throws Exception { + final ConfiguredAirbyteCatalog configuredCatalog = Jsons.clone(CONFIGURED_CATALOG); + // set make stream to full refresh. + configuredCatalog.getStreams().get(0).setSyncMode(SyncMode.FULL_REFRESH); + + final AutoCloseableIterator read1 = source.read(getConfig(PSQL_DB, dbName), configuredCatalog, null); + final List actualRecords1 = AutoCloseableIterators.toListAndClose(read1); + + final Set recordMessages1 = extractRecordMessages(actualRecords1); + final List stateMessages1 = extractStateMessages(actualRecords1); + + assertExpectedStateMessages(stateMessages1); + assertExpectedRecords( + Stream.concat(MAKE_RECORDS.stream(), MODEL_RECORDS.stream()).collect(Collectors.toSet()), + recordMessages1, + Collections.singleton(MODELS_STREAM_NAME)); + + final JsonNode fiatRecord = Jsons.jsonNode(ImmutableMap.of(COL_ID, 3, COL_MAKE, "Fiat")); + final JsonNode puntoRecord = Jsons.jsonNode(ImmutableMap.of(COL_ID, 100, COL_MAKE_ID, 3, COL_MODEL, "Punto")); + database.query(ctx -> { + writeMakeRecord(ctx, fiatRecord); + writeModelRecord(ctx, puntoRecord); + return null; + }); + + final JsonNode state = extractStateMessages(actualRecords1).get(0).getData(); + final AutoCloseableIterator read2 = source.read(getConfig(PSQL_DB, dbName), configuredCatalog, state); + final List actualRecords2 = AutoCloseableIterators.toListAndClose(read2); + + final Set recordMessages2 = extractRecordMessages(actualRecords2); + final List stateMessages2 = extractStateMessages(actualRecords2); + + assertExpectedStateMessages(stateMessages2); + // only make stream should full refresh. + assertExpectedRecords( + Streams.concat(MAKE_RECORDS.stream(), Stream.of(fiatRecord, puntoRecord)).collect(Collectors.toSet()), + recordMessages2, + Collections.singleton(MODELS_STREAM_NAME)); + } + + @Test + @DisplayName("When no records exist, no records are returned.") + void testNoData() throws Exception { + database.query(ctx -> { + ctx.execute(String.format("DELETE FROM %s", MAKES_STREAM_NAME)); + return null; + }); + + database.query(ctx -> { + ctx.execute(String.format("DELETE FROM %s", MODELS_STREAM_NAME)); + return null; + }); + + final AutoCloseableIterator read = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords = AutoCloseableIterators.toListAndClose(read); + + final Set recordMessages = extractRecordMessages(actualRecords); + final List stateMessages = extractStateMessages(actualRecords); + + assertExpectedRecords(Collections.emptySet(), recordMessages); + assertExpectedStateMessages(stateMessages); + } + + @Test + @DisplayName("When no changes have been made to the database since the previous sync, no records are returned.") + void testNoDataOnSecondSync() throws Exception { + final AutoCloseableIterator read1 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + final List actualRecords1 = AutoCloseableIterators.toListAndClose(read1); + final JsonNode state = extractStateMessages(actualRecords1).get(0).getData(); + + final AutoCloseableIterator read2 = source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, state); + final List actualRecords2 = AutoCloseableIterators.toListAndClose(read2); + + final Set recordMessages2 = extractRecordMessages(actualRecords2); + final List stateMessages2 = extractStateMessages(actualRecords2); + + assertExpectedRecords(Collections.emptySet(), recordMessages2); + assertExpectedStateMessages(stateMessages2); + } + + @Test + void testCheck() { + final AirbyteConnectionStatus status = source.check(getConfig(PSQL_DB, dbName)); + assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.SUCCEEDED); + } + + @Test + void testCheckWithoutPublication() throws SQLException { + database.query(ctx -> ctx.execute("DROP PUBLICATION " + PUBLICATION + ";")); + final AirbyteConnectionStatus status = source.check(getConfig(PSQL_DB, dbName)); + assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED); + } + + @Test + void testCheckWithoutReplicationSlot() throws SQLException { + final String fullReplicationSlot = SLOT_NAME_BASE + "_" + dbName; + database.query(ctx -> ctx.execute("SELECT pg_drop_replication_slot('" + fullReplicationSlot + "');")); + + final AirbyteConnectionStatus status = source.check(getConfig(PSQL_DB, dbName)); + assertEquals(status.getStatus(), AirbyteConnectionStatus.Status.FAILED); + } + + @Test + void testReadWithoutPublication() throws SQLException { + database.query(ctx -> ctx.execute("DROP PUBLICATION " + PUBLICATION + ";")); + + assertThrows(Exception.class, () -> { + source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + }); + } + + @Test + void testReadWithoutReplicationSlot() throws SQLException { + final String fullReplicationSlot = SLOT_NAME_BASE + "_" + dbName; + database.query(ctx -> ctx.execute("SELECT pg_drop_replication_slot('" + fullReplicationSlot + "');")); + + assertThrows(Exception.class, () -> { + source.read(getConfig(PSQL_DB, dbName), CONFIGURED_CATALOG, null); + }); + } + + private void writeMakeRecord(DSLContext ctx, JsonNode recordJson) { + ctx.execute(String.format("INSERT INTO %s (%s, %s) VALUES (%s, '%s');", MAKES_STREAM_NAME, COL_ID, COL_MAKE, + recordJson.get(COL_ID).asInt(), recordJson.get(COL_MAKE).asText())); + } + + private void writeModelRecord(DSLContext ctx, JsonNode recordJson) { + ctx.execute(String.format("INSERT INTO %s (%s, %s, %s) VALUES (%s, %s, '%s');", MODELS_STREAM_NAME, COL_ID, COL_MAKE_ID, COL_MODEL, + recordJson.get(COL_ID).asInt(), recordJson.get(COL_MAKE_ID).asInt(), recordJson.get(COL_MODEL).asText())); + } + + private Set extractRecordMessages(List messages) { + final List recordMessageList = messages + .stream() + .filter(r -> r.getType() == Type.RECORD).map(AirbyteMessage::getRecord) + .collect(Collectors.toList()); + final Set recordMessageSet = new HashSet<>(recordMessageList); + + assertEquals(recordMessageList.size(), recordMessageSet.size(), "Expected no duplicates in airbyte record message output for a single sync."); + + return recordMessageSet; + } + + private List extractStateMessages(List messages) { + return messages.stream().filter(r -> r.getType() == Type.STATE).map(AirbyteMessage::getState).collect(Collectors.toList()); + } + + private static void assertExpectedStateMessages(List stateMessages) { + assertEquals(1, stateMessages.size()); + assertNotNull(stateMessages.get(0).getData()); + } + + private static void assertExpectedRecords(Set expectedRecords, Set actualRecords) { + // assume all streams are cdc. + assertExpectedRecords( + expectedRecords, + actualRecords, + actualRecords.stream().map(AirbyteRecordMessage::getStream).collect(Collectors.toSet())); + } + + private static void assertExpectedRecords(Set expectedRecords, Set actualRecords, Set cdcStreams) { + final Set actualData = actualRecords + .stream() + .map(recordMessage -> { + assertTrue(STREAM_NAMES.contains(recordMessage.getStream())); + assertNotNull(recordMessage.getEmittedAt()); + + final JsonNode data = recordMessage.getData(); + + if (cdcStreams.contains(recordMessage.getStream())) { + assertNotNull(data.get(AbstractJdbcSource.CDC_LSN)); + assertNotNull(data.get(AbstractJdbcSource.CDC_UPDATED_AT)); + } else { + assertNull(data.get(AbstractJdbcSource.CDC_LSN)); + assertNull(data.get(AbstractJdbcSource.CDC_UPDATED_AT)); + assertNull(data.get(AbstractJdbcSource.CDC_DELETED_AT)); + } + + ((ObjectNode) data).remove(AbstractJdbcSource.CDC_LSN); + ((ObjectNode) data).remove(AbstractJdbcSource.CDC_UPDATED_AT); + ((ObjectNode) data).remove(AbstractJdbcSource.CDC_DELETED_AT); + + return data; + }) + .collect(Collectors.toSet()); + + assertEquals(expectedRecords, actualData); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumEventUtilsTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumEventUtilsTest.java new file mode 100644 index 000000000000..d7b846adc506 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumEventUtilsTest.java @@ -0,0 +1,88 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.when; + +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.protocol.models.AirbyteMessage; +import io.airbyte.protocol.models.AirbyteRecordMessage; +import io.debezium.engine.ChangeEvent; +import java.io.IOException; +import java.time.Instant; +import org.junit.jupiter.api.Test; + +class DebeziumEventUtilsTest { + + @Test + public void testConvertChangeEvent() throws IOException { + final String stream = "public.names"; + final Instant emittedAt = Instant.now(); + ChangeEvent insertChangeEvent = mockChangeEvent("insert_change_event.json"); + ChangeEvent updateChangeEvent = mockChangeEvent("update_change_event.json"); + ChangeEvent deleteChangeEvent = mockChangeEvent("delete_change_event.json"); + + final AirbyteMessage actualInsert = DebeziumEventUtils.toAirbyteMessage(insertChangeEvent, emittedAt); + final AirbyteMessage actualUpdate = DebeziumEventUtils.toAirbyteMessage(updateChangeEvent, emittedAt); + final AirbyteMessage actualDelete = DebeziumEventUtils.toAirbyteMessage(deleteChangeEvent, emittedAt); + + final AirbyteMessage expectedInsert = createAirbyteMessage(stream, emittedAt, "insert_message.json"); + final AirbyteMessage expectedUpdate = createAirbyteMessage(stream, emittedAt, "update_message.json"); + final AirbyteMessage expectedDelete = createAirbyteMessage(stream, emittedAt, "delete_message.json"); + + deepCompare(expectedInsert, actualInsert); + deepCompare(expectedUpdate, actualUpdate); + deepCompare(expectedDelete, actualDelete); + } + + private static ChangeEvent mockChangeEvent(String resourceName) throws IOException { + final ChangeEvent mocked = mock(ChangeEvent.class); + final String resource = MoreResources.readResource(resourceName); + when(mocked.value()).thenReturn(resource); + + return mocked; + } + + private static AirbyteMessage createAirbyteMessage(String stream, Instant emittedAt, String resourceName) throws IOException { + final String data = MoreResources.readResource(resourceName); + + final AirbyteRecordMessage recordMessage = new AirbyteRecordMessage() + .withStream(stream) + .withData(Jsons.deserialize(data)) + .withEmittedAt(emittedAt.toEpochMilli()); + + return new AirbyteMessage() + .withType(AirbyteMessage.Type.RECORD) + .withRecord(recordMessage); + } + + private static void deepCompare(Object expected, Object actual) { + assertEquals(Jsons.deserialize(Jsons.serialize(expected)), Jsons.deserialize(Jsons.serialize(actual))); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisherTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisherTest.java new file mode 100644 index 000000000000..56804df77eb5 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/DebeziumRecordPublisherTest.java @@ -0,0 +1,62 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import static org.junit.jupiter.api.Assertions.*; + +import com.google.common.collect.ImmutableList; +import io.airbyte.protocol.models.CatalogHelpers; +import io.airbyte.protocol.models.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.SyncMode; +import org.junit.jupiter.api.Test; + +class DebeziumRecordPublisherTest { + + @Test + public void testWhitelistCreation() { + final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(ImmutableList.of( + CatalogHelpers.createConfiguredAirbyteStream("public.id_and_name").withSyncMode(SyncMode.INCREMENTAL), + CatalogHelpers.createConfiguredAirbyteStream("public.id_,something").withSyncMode(SyncMode.INCREMENTAL), + CatalogHelpers.createConfiguredAirbyteStream("public.n\"aMéS").withSyncMode(SyncMode.INCREMENTAL))); + + final String expectedWhitelist = "public.id_and_name,public.id_\\,something,public.n\"aMéS"; + final String actualWhitelist = DebeziumRecordPublisher.getTableWhitelist(catalog); + + assertEquals(expectedWhitelist, actualWhitelist); + } + + @Test + public void testWhitelistFiltersFullRefresh() { + final ConfiguredAirbyteCatalog catalog = new ConfiguredAirbyteCatalog().withStreams(ImmutableList.of( + CatalogHelpers.createConfiguredAirbyteStream("public.id_and_name").withSyncMode(SyncMode.INCREMENTAL), + CatalogHelpers.createConfiguredAirbyteStream("public.id_and_name2").withSyncMode(SyncMode.FULL_REFRESH))); + + final String expectedWhitelist = "public.id_and_name"; + final String actualWhitelist = DebeziumRecordPublisher.getTableWhitelist(catalog); + + assertEquals(expectedWhitelist, actualWhitelist); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSourceTest.java index 3d58b9188bde..cf99e6a29498 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSourceTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSourceTest.java @@ -25,9 +25,11 @@ package io.airbyte.integrations.source.postgres; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -220,6 +222,18 @@ void testReadSuccess() throws Exception { assertEquals(ASCII_MESSAGES, actualMessages); } + @Test + void testIsCdc() { + final JsonNode config = getConfig(PSQL_DB, dbName); + + assertFalse(PostgresSource.isCdc(config)); + + ((ObjectNode) config).set("replication_method", Jsons.jsonNode(ImmutableMap.of( + "replication_slot", "slot", + "publication", "ab_pub"))); + assertTrue(PostgresSource.isCdc(config)); + } + private static AirbyteMessage createRecord(String stream, Map data) { return new AirbyteMessage().withType(Type.RECORD).withRecord(new AirbyteRecordMessage().withData(Jsons.jsonNode(data)).withStream(stream)); } diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSpecTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSpecTest.java new file mode 100644 index 000000000000..69dfd75190c1 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/PostgresSpecTest.java @@ -0,0 +1,125 @@ +/* + * MIT License + * + * Copyright (c) 2020 Airbyte + * + * Permission is hereby granted, free of charge, to any person obtaining a copy + * of this software and associated documentation files (the "Software"), to deal + * in the Software without restriction, including without limitation the rights + * to use, copy, modify, merge, publish, distribute, sublicense, and/or sell + * copies of the Software, and to permit persons to whom the Software is + * furnished to do so, subject to the following conditions: + * + * The above copyright notice and this permission notice shall be included in all + * copies or substantial portions of the Software. + * + * THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR + * IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, + * FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE + * AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER + * LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, + * OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN THE + * SOFTWARE. + */ + +package io.airbyte.integrations.source.postgres; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.airbyte.commons.io.IOs; +import io.airbyte.commons.json.Jsons; +import io.airbyte.commons.resources.MoreResources; +import io.airbyte.validation.json.JsonSchemaValidator; +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; + +/** + * Tests that the postgres spec passes JsonSchema validation. While this may seem like overkill, we + * are doing it because there are some gotchas in correctly configuring the oneOf. + */ +public class PostgresSpecTest { + + private static final String CONFIGURATION = "{ " + + "\"password\" : \"pwd\", " + + "\"username\" : \"postgres\", " + + "\"database\" : \"postgres_db\", " + + "\"port\" : 5432, " + + "\"host\" : \"localhost\", " + + "\"replication_method\" : { \"replication_slot\" : \"ab_slot\", \"publication\" : \"ab_publication\" }" + + "}"; + private static JsonNode schema; + private static JsonSchemaValidator validator; + + @BeforeAll + static void init() throws IOException { + final String spec = MoreResources.readResource("spec.json"); + final File schemaFile = IOs.writeFile(Files.createTempDirectory(Path.of("/tmp"), "pg-spec-test"), "schema.json", spec).toFile(); + schema = JsonSchemaValidator.getSchema(schemaFile).get("connectionSpecification"); + validator = new JsonSchemaValidator(); + } + + @Test + void testDatabaseMissing() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config).remove("database"); + assertFalse(validator.test(schema, config)); + } + + @Test + void testWithoutReplicationMethod() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config).remove("replication_method"); + + assertTrue(validator.test(schema, config)); + } + + @Test + void testWithReplicationMethodWithReplicationSlot() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + assertTrue(validator.test(schema, config)); + } + + @Test + void testWithReplicationMethodMissingPublication() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config.get("replication_method")).remove("replication_slot"); + + assertFalse(validator.test(schema, config)); + } + + @Test + void testWithReplicationMethodStandard() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config.get("replication_method")).remove("replication_slot"); + ((ObjectNode) config.get("replication_method")).remove("publication"); + assertTrue(validator.test(schema, config)); + + final JsonNode configReplicationMethodNotSet = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) configReplicationMethodNotSet).remove("replication_method"); + assertTrue(validator.test(schema, configReplicationMethodNotSet)); + } + + @Test + void testWithReplicationMethodWithReplicationSlotWithWrongType() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config.get("replication_method")).put("replication_slot", 10); + + assertFalse(validator.test(schema, config)); + } + + @Test + void testWithReplicationMethodWithReplicationSlotWithNull() { + final JsonNode config = Jsons.deserialize(CONFIGURATION); + ((ObjectNode) config.get("replication_method")).set("replication_slot", null); + + assertFalse(validator.test(schema, config)); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_change_event.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_change_event.json new file mode 100644 index 000000000000..07b575bf7e2c --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_change_event.json @@ -0,0 +1,25 @@ +{ + "before": { + "first_name": "san", + "last_name": "goku", + "power": null + }, + "after": null, + "source": { + "version": "1.4.2.Final", + "connector": "postgresql", + "name": "orders", + "ts_ms": 1616775646886, + "snapshot": false, + "db": "db_lwfoyffqvx", + "schema": "public", + "table": "names", + "txId": 498, + "lsn": 23012360, + "xmin": null + }, + "op": "d", + "ts_ms": 1616775646931, + "transaction": null, + "destination": "orders.public.names" +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_message.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_message.json new file mode 100644 index 000000000000..a14eab66fe17 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/delete_message.json @@ -0,0 +1,8 @@ +{ + "first_name": "san", + "last_name": "goku", + "power": null, + "_ab_cdc_updated_at": 1616775646886, + "_ab_cdc_lsn": 23012360, + "_ab_cdc_deleted_at": 1616775646886 +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_change_event.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_change_event.json new file mode 100644 index 000000000000..4b2c2fb6e2cf --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_change_event.json @@ -0,0 +1,25 @@ +{ + "before": null, + "after": { + "first_name": "san", + "last_name": "goku", + "power": "Infinity" + }, + "source": { + "version": "1.4.2.Final", + "connector": "postgresql", + "name": "orders", + "ts_ms": 1616775642623, + "snapshot": true, + "db": "db_lwfoyffqvx", + "schema": "public", + "table": "names", + "txId": 495, + "lsn": 23011544, + "xmin": null + }, + "op": "r", + "ts_ms": 1616775642624, + "transaction": null, + "destination": "orders.public.names" +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_message.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_message.json new file mode 100644 index 000000000000..46abad6a267a --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/insert_message.json @@ -0,0 +1,8 @@ +{ + "first_name": "san", + "last_name": "goku", + "power": "Infinity", + "_ab_cdc_updated_at": 1616775642623, + "_ab_cdc_lsn": 23011544, + "_ab_cdc_deleted_at": null +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/test_debezium_offset.dat b/airbyte-integrations/connectors/source-postgres/src/test/resources/test_debezium_offset.dat new file mode 100644 index 000000000000..c7e7054916ed Binary files /dev/null and b/airbyte-integrations/connectors/source-postgres/src/test/resources/test_debezium_offset.dat differ diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/update_change_event.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/update_change_event.json new file mode 100644 index 000000000000..da5dcd9c2b06 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/update_change_event.json @@ -0,0 +1,25 @@ +{ + "before": null, + "after": { + "first_name": "san", + "last_name": "goku", + "power": 10000.2 + }, + "source": { + "version": "1.4.2.Final", + "connector": "postgresql", + "name": "orders", + "ts_ms": 1616775646881, + "snapshot": false, + "db": "db_lwfoyffqvx", + "schema": "public", + "table": "names", + "txId": 497, + "lsn": 23012216, + "xmin": null + }, + "op": "u", + "ts_ms": 1616775646929, + "transaction": null, + "destination": "orders.public.names" +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/resources/update_message.json b/airbyte-integrations/connectors/source-postgres/src/test/resources/update_message.json new file mode 100644 index 000000000000..757c5833d253 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/resources/update_message.json @@ -0,0 +1,8 @@ +{ + "first_name": "san", + "last_name": "goku", + "power": 10000.2, + "_ab_cdc_updated_at": 1616775646881, + "_ab_cdc_lsn": 23012216, + "_ab_cdc_deleted_at": null +} diff --git a/docs/SUMMARY.md b/docs/SUMMARY.md index e4290ce31191..1108ca4f6a74 100644 --- a/docs/SUMMARY.md +++ b/docs/SUMMARY.md @@ -91,6 +91,7 @@ * [High-level View](architecture/high-level-view.md) * [Workers & Jobs](architecture/jobs.md) * [Technical Stack](architecture/tech-stack.md) + * [Change Data Capture (CDC)](architecture/cdc.md) * [Contributing to Airbyte](contributing-to-airbyte/README.md) * [Code of Conduct](contributing-to-airbyte/code-of-conduct.md) * [Developing Locally](contributing-to-airbyte/developing-locally.md) diff --git a/docs/architecture/cdc.md b/docs/architecture/cdc.md new file mode 100644 index 000000000000..5639e526946d --- /dev/null +++ b/docs/architecture/cdc.md @@ -0,0 +1,33 @@ +# Change Data Capture (CDC) + +## What is log-based incremental replication? +Many common databases support writing all record changes to log files for the purpose of replication. A consumer of these log files (such as Airbyte) can read these logs while keeping track of the current position within the logs in order to read all record changes coming from `DELETE`/`INSERT`/`UPDATE` statements. + +## Syncing +The orchestration for syncing is similar to non-CDC database sources. After selecting a sync interval, syncs are launched regularly. We read data from the log up to the time that the sync was started. We do not treat CDC sources as infinite streaming sources. You should ensure that your schedule for running these syncs is frequent enough to consume the logs that are generated. The first time the sync is run, a snapshot of the current state of the data will be taken. This is done using `SELECT` statements and is effectively a Full Refresh. Subsequent syncs will use the logs to determine which changes took place since the last sync and update those. Airbyte keeps track of the current log position between syncs. + +A single sync might have some tables configured for Full Refresh replication and others for Incremental. If CDC is configured at the source level, all tables with Incremental selected will use CDC. All Full Refresh tables will replicate using the same process as non-CDC sources. However, these tables will still include CDC metadata columns by default. + +The Airbyte Protocol outputs records from sources. Records from `UPDATE` statements appear the same way as records from `INSERT` statements. We support different options for how to sync this data into destinations using primary keys, so you can choose to append this data, delete in place, etc. + +We add some metadata columns for CDC sources: +* `ab_cdc_lsn` is the point in the log where the record was retrieved +* `ab_cdc_updated_at` is the timestamp for the database transaction that resulted in this record change and is present for records from `DELETE`/`INSERT`/`UPDATE` statements +* `ab_cdc_deleted_at` is the timestamp for the database transaction that resulted in this record change and is only present for records from `DELETE` statements + +## Limitations +* CDC incremental is only supported for tables with primary keys. A CDC source can still choose to replicate tables without primary keys as Full Refresh or a non-CDC source can be configured for the same database to replicate the tables without primary keys using standard incremental replication. +* Data must be in tables, not views. +* The modifications you are trying to capture must be made using `DELETE`/`INSERT`/`UPDATE`. For example, changes made from `TRUNCATE`/`ALTER` won't appear in logs and therefore in your destination. +* We do not support schema changes automatically for CDC sources. We recommend resetting and resyncing data if you make a schema change. +* There are database-specific limitations. See the documentation pages for individual connectors for more information. +* The records produced by `DELETE` statements only contain primary keys. All other data fields are unset. + +## Current Support +* [Postgres](../integrations/sources/postgres.md) + +## Coming Soon +* [MySQL](../integrations/sources/mysql.md) +* [SQL Server / MSSQL](../integrations/sources/mssql.md) +* Oracle DB +* Please [create a ticket](https://github.com/airbytehq/airbyte/issues/new/choose) if you need CDC support on another database! \ No newline at end of file diff --git a/docs/faq/technical-support.md b/docs/faq/technical-support.md index 5414277ac676..b5025bbb86f5 100644 --- a/docs/faq/technical-support.md +++ b/docs/faq/technical-support.md @@ -71,6 +71,10 @@ Depending on your Docker network configuration, you may not be able to connect t If you are running into connection refused errors when running Airbyte via Docker Compose on Mac, try using `host.docker.internal` as the host. On Linux, you may have to modify `docker-compose.yml` and add a host that maps to your local machine using [`extra_hosts`](https://docs.docker.com/compose/compose-file/compose-file-v3/#extra_hosts). +## **Do you support change data capture (CDC) or logical replication for databases?** + +We currently support [CDC for Postgres 10+](../integrations/sources/postgres.md). We are adding support for a few other databases April/May 2021. + ## **Can I disable analytics in Airbyte?** Yes, you can control what's sent outside of Airbyte for analytics purposes. diff --git a/docs/integrations/sources/postgres.md b/docs/integrations/sources/postgres.md index 1b6f1055bb2e..b7f057f331aa 100644 --- a/docs/integrations/sources/postgres.md +++ b/docs/integrations/sources/postgres.md @@ -51,8 +51,8 @@ Postgres data types are mapped to the following data types when synchronizing da | :--- | :--- | | Full Refresh Sync | Yes | | Incremental - Append Sync | Yes | -| Replicate Incremental Deletes | Coming soon | -| Logical Replication \(WAL\) | Coming soon | +| Replicate Incremental Deletes | Yes | +| Logical Replication \(WAL\) | Yes | | SSL Support | Yes | | SSH Tunnel Connection | Coming soon | @@ -97,5 +97,106 @@ GRANT SELECT ON ALL TABLES IN SCHEMA TO airbyte; ALTER DEFAULT PRIVILEGES IN SCHEMA GRANT SELECT ON TABLES TO airbyte; ``` +#### 3. Set up CDC \(Optional\) + +Please read [the section on CDC below](#setting-up-cdc-for-postgres) for more information. + +#### 4. That's it! + Your database user should now be ready for use with Airbyte. +## Change Data Capture (CDC) / Logical Replication / WAL Replication +We use [logical replication](https://www.postgresql.org/docs/10/logical-replication.html) of the Postgres write-ahead log (WAL) to incrementally capture deletes using the `pgoutput` plugin. + +We do not require installing custom plugins like `wal2json` or `test_decoding`. We use `pgoutput`, which is included in Postgres 10+ by default. + +Please read the [CDC docs](../../architecture/cdc.md) for an overview of how Airbyte approaches CDC. + +### Should I use CDC for Postgres? +* If you need a record of deletions and can accept the limitations posted below, you should to use CDC for Postgres. +* If your data set is small and you just want snapshot of your table in the destination, consider using Full Refresh replication for your table instead of CDC. +* If the limitations prevent you from using CDC and your goal is to maintain a snapshot of your table in the destination, consider using non-CDC incremental and occasionally reset the data and re-sync. +* If your table has a primary key but doesn't have a reasonable cursor field for incremental syncing (i.e. `updated_at`), CDC allows you to sync your table incrementally. + +### CDC Limitations +* Make sure to read our [CDC docs](../../architecture/cdc.md) to see limitations that impact all databases using CDC replication. +* CDC is only available for Postgres 10+. +* Airbyte requires a replication slot configured only for its use. Only one source should be configured that uses this replication slot. Instructions on how to set up a replication slot can be found below. +* Log-based replication only works for master instances of Postgres. +* Using logical replication increases disk space used on the database server. The additional data is stored until it is consumed. + * We recommend setting frequent syncs for CDC in order to ensure that this data doesn't fill up your disk space. + * If you stop syncing a CDC-configured Postgres instance to Airbyte, you should delete the replication slot. Otherwise, it may fill up your disk space. +* Our CDC implementation uses at least once delivery for all change records. + +### Setting up CDC for Postgres + +#### Enable logical replication + +Follow one of these guides to enable logical replication: +* [Bare Metal, VMs (EC2/GCE/etc), Docker, etc.](#setting-up-cdc-on-bare-metal-vms-ec2gceetc-docker-etc) +* [AWS Postgres RDS or Aurora](#setting-up-cdc-on-aws-postgres-rds-or-aurora) +* [Azure Database for Postgres](#setting-up-cdc-on-azure-database-for-postgres) + +#### Add user-level permissions + +We recommend using a user specifically for Airbyte's replication so you can minimize access. This Airbyte user for your instance needs to be granted `REPLICATION` and `LOGIN` permissions. You can create a role with `CREATE ROLE REPLICATION LOGIN;` and grant that role to the user. You still need to make sure the user can connect to the database, use the schema, and to use `SELECT` on tables (the same are required for non-CDC incremental syncs and all full refreshes). + +#### Create replication slot +Next, you will need to create a replication slot. Here is the query used to create a replication slot called `airbyte_slot`: +``` +SELECT pg_create_logical_replication_slot('airbyte_slot', 'pgoutput');` +``` + +This slot **must** use `pgoutput`. + +#### Create publications and replication identities for tables + +For each table you want to replicate with CDC, you will need to run `CREATE PUBLICATION airbyte_publication FOR TABLES ;`. This publication name is customizable. For each of these tables, you will need to run `ALTER TABLE tbl1 REPLICA IDENTITY DEFAULT;`. **You cannot run `ALTER`/`UPDATE`/`DELETE` commands on a table between the creation of a publication and adding the replica identity**, so we recommend running the `CREATE PUBLICATION` and adding all relevant `REPLICATION IDENTITY` alterations immediately. Please refer to the [Postgres docs](https://www.postgresql.org/docs/10/sql-alterpublication.html) if you need to add or remove tables from your publication in the future. + +The UI currently allows selecting any tables for CDC. If a table is selected that is not part of the publication, it will not replicate even though it is selected. If a table is part of the publication but does not have a replication identity, that replication identity will be created automatically on the first run if the Airbyte user has the necessary permissions. + +#### Start syncing +When configuring the source, select CDC and provide the replication slot and publication you just created. You should be ready to sync data with CDC! + +### Setting up CDC on Bare Metal, VMs (EC2/GCE/etc), Docker, etc. +Some settings must be configured in the `postgresql.conf` file for your database. You can find the location of this file using `psql -U postgres -c 'SHOW config_file'` withe the correct `psql` credentials specified. Alternatively, a custom file can be specified when running postgres with the `-c` flag. For example `postgres -c config_file=/etc/postgresql/postgresql.conf` runs Postgres with the config file at `/etc/postgresql/postgresql.conf`. + +If you are syncing data from a server using the `postgres` Docker image, you will need to mount a file and change the command to run Postgres with the set config file. If you're just testing CDC behavior, you may want to use a modified version of a [sample `postgresql.conf`](https://github.com/postgres/postgres/blob/master/src/backend/utils/misc/postgresql.conf.sample). + +* `wal_level` is the type of coding used within the Postgres write-ahead log. This must be set to `logical` for Airbyte CDC. +* `max_wal_senders` is the maximum number of processes used for handling WAL changes. This must be at least one. +* `max_replication_slots` is the maximum number of replication slots that are allowed to stream WAL changes. This must one if Airbyte will be the only service reading subscribing to WAL changes or more if other services are also reading from the WAL. + +Here is what these settings would look like in `postgresql.conf`: +``` +wal_level = logical +max_wal_senders = 1 +max_replication_slots = 1 +``` + +After setting these values you will need to restart your instance. + +Finally, [follow the rest of steps above](#setting-up-cdc-for-postgres). + +### Setting up CDC on AWS Postgres RDS or Aurora +* Go to the `Configuration` tab for your DB cluster. +* Find your cluster parameter group. You will either edit the parameters for this group or create a copy of this parameter group to edit. If you create a copy you will need to change your cluster's parameter group before restarting. +* Within the parameter group page, search for `rds.logical_replication`. Select this row and click on the `Edit parameters` button. Set this value to `1`. +* Wait for a maintenance window to automatically restart the instance or restart it manually. +* Finally, [follow the rest of steps above](#setting-up-cdc-for-postgres). + +### Setting up CDC on Azure Database for Postgres +Use either the Azure CLI to: +``` +az postgres server configuration set --resource-group group --server-name server --name azure.replication_support --value logical +az postgres server restart --resource-group group --name server +``` + +Finally, [follow the rest of steps above](#setting-up-cdc-for-postgres). + +### Setting up CDC on Google CloudSQL + +Unfortunately, logical replication is not configurable for Google CloudSQL. You can indicate your support for this feature on the [Google Issue Tracker](https://issuetracker.google.com/issues/120274585). + +### Setting up CDC on other platforms +If you encounter one of those not listed below, please consider [contributing to our docs](https://github.com/airbytehq/airbyte/tree/master/docs) and providing setup instructions.