From 194a62cf89bea4ae7bd0f4cd39500161936ee34a Mon Sep 17 00:00:00 2001 From: Rodi Reich Zilberman <867491+rodireich@users.noreply.github.com> Date: Fri, 9 Jun 2023 22:49:54 -0700 Subject: [PATCH 01/27] initial ctid for testing --- .../source/postgres/PostgresSource.java | 7 +- .../postgres/ctid/CtidStateIterator.java | 69 +++++++ .../postgres/ctid/CtidStateManager.java | 76 ++++++++ .../postgres/ctid/PostgresCtidHandler.java | 175 ++++++++++++++++++ .../internal_models/ctid_internal_models.yaml | 15 ++ 5 files changed, 341 insertions(+), 1 deletion(-) create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml 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 d7fd2e43e2d56c..6b0599ef1cef30 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 @@ -56,6 +56,8 @@ import io.airbyte.integrations.source.jdbc.JdbcSSLConnectionUtils; import io.airbyte.integrations.source.jdbc.JdbcSSLConnectionUtils.SslMode; import io.airbyte.integrations.source.jdbc.dto.JdbcPrivilegeDto; +import io.airbyte.integrations.source.postgres.ctid.CtidStateManager; +import io.airbyte.integrations.source.postgres.ctid.PostgresCtidHandler; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; import io.airbyte.integrations.source.postgres.xmin.PostgresXminHandler; import io.airbyte.integrations.source.postgres.xmin.XminStateManager; @@ -456,7 +458,10 @@ public List> getIncrementalIterators(final final PostgresXminHandler handler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); return handler.getIncrementalIterators(catalog, tableNameToTable, emittedAt); } else { - return super.getIncrementalIterators(database, catalog, tableNameToTable, stateManager, emittedAt); + final CtidStateManager ctidStateManager = new CtidStateManager(stateManager.getRawStateMessages()); + final PostgresCtidHandler handler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), null, ctidStateManager); + return handler.getIncrementalIterators(catalog, tableNameToTable, emittedAt); +// return super.getIncrementalIterators(database, catalog, tableNameToTable, stateManager, emittedAt); } } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java new file mode 100644 index 00000000000000..ff20f271ad4b87 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -0,0 +1,69 @@ +package io.airbyte.integrations.source.postgres.ctid; + +import com.google.common.collect.AbstractIterator; +import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.v0.AirbyteMessage; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicLong; +import javax.annotation.CheckForNull; +import org.apache.commons.lang3.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CtidStateIterator extends AbstractIterator implements Iterator +{ + private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateIterator.class); + private final Iterator messageIterator; + private final AirbyteStreamNameNamespacePair pair; + private boolean hasEmittedFinalState; + private boolean hasCaughtException = false; + private String lastCtid; + final AtomicLong recordCount = new AtomicLong(); + public CtidStateIterator(final Iterator messageIterator, + final AirbyteStreamNameNamespacePair pair) { + this.messageIterator = messageIterator; + this.pair = pair; + } + + @CheckForNull + @Override + protected AirbyteMessage computeNext() { + final long count = recordCount.incrementAndGet(); + if (hasCaughtException) { + // Mark iterator as done since the next call to messageIterator will result in an + // IllegalArgumentException and resets exception caught state. + // This occurs when the previous iteration emitted state so this iteration cycle will indicate + // iteration is complete + hasCaughtException = false; + return endOfData(); + } + + if (messageIterator.hasNext()) { + if (count % 10_000 == 0 && StringUtils.isNotBlank(lastCtid)) { + LOGGER.info("saving ctid state with {}", this.lastCtid); + return CtidStateManager.createStateMessage(pair, new CtidStatus().withCtid(lastCtid)); + } + // Use try-catch to catch Exception that could occur when connection to the database fails + try { + final AirbyteMessage message = messageIterator.next(); + if (message.getRecord().getData().hasNonNull("ctid")) { + this.lastCtid = message.getRecord().getData().get("ctid").asText(); + } + return message; + } catch (final Exception e) { + hasCaughtException = true; + LOGGER.error("Message iterator failed to read next record.", e); + // We want to still continue attempting to sync future streams, so the exception is caught. When + // frequent state emission is introduced, this + // will result in a partial success. + return endOfData(); + } + } else if (!hasEmittedFinalState) { + hasEmittedFinalState = true; + return CtidStateManager.createStateMessage(pair, new CtidStatus().withCtid(lastCtid)); + } else { + return endOfData(); + } + } +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java new file mode 100644 index 00000000000000..406a24e80f39ff --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -0,0 +1,76 @@ +package io.airbyte.integrations.source.postgres.ctid; + +import com.fasterxml.jackson.databind.ObjectMapper; +import io.airbyte.commons.exceptions.ConfigErrorException; +import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.Jsons; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; +import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; +import io.airbyte.protocol.models.v0.StreamDescriptor; +import io.airbyte.protocol.models.v0.AirbyteMessage; +import io.airbyte.protocol.models.v0.AirbyteMessage.Type; +import io.airbyte.protocol.models.v0.AirbyteStreamState; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CtidStateManager { + + private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateManager.class); + + private final Map pairToCtidStatus; + private final static AirbyteStateMessage EMPTY_STATE = new AirbyteStateMessage() + .withType(AirbyteStateType.STREAM) + .withStream(new AirbyteStreamState()); + + public CtidStateManager(final List stateMessages) { + this.pairToCtidStatus = createPairToCtidStatusMap(stateMessages); + } + + private static Map createPairToCtidStatusMap(final List stateMessages) { + final Map localMap = new HashMap<>(); + if (stateMessages != null) { + for (final AirbyteStateMessage stateMessage : stateMessages) { + if (stateMessage.getType() == AirbyteStateType.STREAM && !stateMessage.equals(EMPTY_STATE)) { + LOGGER.info("State message: " + stateMessage); + final StreamDescriptor streamDescriptor = stateMessage.getStream().getStreamDescriptor(); + final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace()); + final CtidStatus ctidStatus; + try { + ctidStatus = Jsons.object(stateMessage.getStream().getStreamState(), CtidStatus.class); + } catch (final IllegalArgumentException e) { + throw new ConfigErrorException("Invalid per-stream state"); + } + localMap.put(pair, ctidStatus); + } + } + } + return localMap; + } + + public CtidStatus getCtidStatus(final AirbyteStreamNameNamespacePair pair) { + return pairToCtidStatus.get(pair); + } + + public static AirbyteMessage createStateMessage(final AirbyteStreamNameNamespacePair pair, final CtidStatus ctidStatus) { + final AirbyteStreamState airbyteStreamState = + new AirbyteStreamState() + .withStreamDescriptor( + new StreamDescriptor() + .withName(pair.getName()) + .withNamespace(pair.getNamespace())) + .withStreamState(new ObjectMapper().valueToTree(ctidStatus)); + + final AirbyteStateMessage stateMessage = + new AirbyteStateMessage() + .withType(AirbyteStateType.STREAM) + .withStream(airbyteStreamState); + + return new AirbyteMessage() + .withType(Type.STATE) + .withState(stateMessage); + } +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java new file mode 100644 index 00000000000000..f199c3230fe86f --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -0,0 +1,175 @@ +package io.airbyte.integrations.source.postgres.ctid; + +import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; + +import com.fasterxml.jackson.databind.JsonNode; +import io.airbyte.commons.stream.AirbyteStreamUtils; +import io.airbyte.commons.util.AutoCloseableIterator; +import io.airbyte.commons.util.AutoCloseableIterators; +import io.airbyte.db.JdbcCompatibleSourceOperations; +import io.airbyte.db.jdbc.JdbcDatabase; +import io.airbyte.integrations.source.postgres.PostgresType; +import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.integrations.source.relationaldb.DbSourceDiscoverUtil; +import io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils; +import io.airbyte.integrations.source.relationaldb.TableInfo; +import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.CommonField; +import io.airbyte.protocol.models.v0.AirbyteMessage; +import io.airbyte.protocol.models.v0.AirbyteMessage.Type; +import io.airbyte.protocol.models.v0.AirbyteRecordMessage; +import io.airbyte.protocol.models.v0.AirbyteStream; +import io.airbyte.protocol.models.v0.CatalogHelpers; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.SyncMode; +import java.sql.Connection; +import java.sql.PreparedStatement; +import java.sql.SQLException; +import java.time.Instant; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.concurrent.atomic.AtomicLong; +import java.util.stream.Stream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class PostgresCtidHandler { + + private static final Logger LOGGER = LoggerFactory.getLogger(PostgresCtidHandler.class); + private final JdbcDatabase database; + private final JdbcCompatibleSourceOperations sourceOperations; + private final String quoteString; + private final CtidStatus ctidStatus; + private final CtidStateManager ctidStateManager; + public PostgresCtidHandler(final JdbcDatabase database, + final JdbcCompatibleSourceOperations sourceOperations, + final String quoteString, + final CtidStatus ctidStatus, + final CtidStateManager ctidStateManager) { + this.database = database; + this.sourceOperations = sourceOperations; + this.quoteString = quoteString; + this.ctidStatus = ctidStatus; + this.ctidStateManager = ctidStateManager; + } + public List> getIncrementalIterators( + final ConfiguredAirbyteCatalog catalog, + final Map>> tableNameToTable, + final Instant emmitedAt) { + final List> iteratorList = new ArrayList<>(); + for (final ConfiguredAirbyteStream airbyteStream : catalog.getStreams()) { + final AirbyteStream stream = airbyteStream.getStream(); + final String streamName = stream.getName(); + final String namespace = stream.getNamespace(); + final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamName, namespace); + final String fullyQualifiedTableName = DbSourceDiscoverUtil.getFullyQualifiedTableName(namespace, streamName); + //TODO: what if stream doesn't need ctid + if (!tableNameToTable.containsKey(fullyQualifiedTableName)) { + LOGGER.info("Skipping stream {} because it is not in the source", fullyQualifiedTableName); + continue; + } + if (airbyteStream.getSyncMode().equals(SyncMode.INCREMENTAL)) { + // Grab the selected fields to sync + final TableInfo> table = tableNameToTable + .get(fullyQualifiedTableName); + final List selectedDatabaseFields = table.getFields() + .stream() + .map(CommonField::getName) + .filter(CatalogHelpers.getTopLevelFieldNames(airbyteStream)::contains) + .toList(); + final AutoCloseableIterator queryStream = queryTableCtid(selectedDatabaseFields, table.getNameSpace(), table.getName()); + final AutoCloseableIterator recordIterator = getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); + final AutoCloseableIterator recordAndMessageIterator = augmentWithState(recordIterator, pair); + iteratorList.add(augmentWithLogs(recordAndMessageIterator, pair, streamName)); + } + } + return iteratorList; + } + + private AutoCloseableIterator queryTableCtid( + final List columnNames, + final String schemaName, + final String tableName) { + + LOGGER.info("Queueing query for table: {}", tableName); + final AirbyteStreamNameNamespacePair airbyteStream = + AirbyteStreamUtils.convertFromNameAndNamespace(tableName, schemaName); + return AutoCloseableIterators.lazyIterator(() -> { + try { + final Stream stream = database.unsafeQuery( + connection -> createCtidQueryStatement(connection, columnNames, schemaName, tableName, airbyteStream), sourceOperations::rowToJson); + return AutoCloseableIterators.fromStream(stream, airbyteStream); + } catch (final SQLException e) { + throw new RuntimeException(e); + } + }, airbyteStream); + } + + private PreparedStatement createCtidQueryStatement( + final Connection connection, + final List columnNames, + final String schemaName, + final String tableName, + final AirbyteStreamNameNamespacePair airbyteStream) { + try { + LOGGER.info("Preparing query for table: {}", tableName); + final String fullTableName = getFullyQualifiedTableNameWithQuoting(schemaName, tableName, + quoteString); + + final String wrappedColumnNames = RelationalDbQueryUtils.enquoteIdentifierList(columnNames, quoteString); + // The xmin state that we save represents the lowest XID that is still in progress. To make sure we + // don't miss + // data associated with the current transaction, we have to issue an >= + final String sql = "SELECT ctid, %s FROM %s WHERE ctid > ?::tid".formatted(wrappedColumnNames, fullTableName); + final PreparedStatement preparedStatement = connection.prepareStatement(sql); + final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(airbyteStream); + if (currentCtidStatus != null) { + preparedStatement.setObject(1, currentCtidStatus.getCtid()); + } else { + preparedStatement.setObject(1, "(0,0)"); + } + LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement); + return preparedStatement; + } catch (final SQLException e) { + throw new RuntimeException(e); + } + } + + // Transforms the given iterator to create an {@link AirbyteRecordMessage} + private static AutoCloseableIterator getRecordIterator( + final AutoCloseableIterator recordIterator, + final String streamName, + final String namespace, + final long emittedAt) { + return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessage() + .withType(Type.RECORD) + .withRecord(new AirbyteRecordMessage() + .withStream(streamName) + .withNamespace(namespace) + .withEmittedAt(emittedAt) + .withData(r))); + } + + // Augments the given iterator with record count logs. + private AutoCloseableIterator augmentWithLogs(final AutoCloseableIterator iterator, + final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair, + final String streamName) { + final AtomicLong recordCount = new AtomicLong(); + return AutoCloseableIterators.transform(iterator, + AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()), + r -> { + final long count = recordCount.incrementAndGet(); + if (count % 10000 == 0) { + LOGGER.info("Reading stream {}. Records read: {}", streamName, count); + } + return r; + }); + } + + private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, final AirbyteStreamNameNamespacePair pair) { + return AutoCloseableIterators.transform( + autoClosableIterator -> new CtidStateIterator(recordIterator, pair), recordIterator, pair); + } +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml new file mode 100644 index 00000000000000..3c6cf97b9efaf6 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml @@ -0,0 +1,15 @@ +--- +"$schema": http://json-schema.org/draft-07/schema# +title: Postgres Models +type: object +description: Postgres Models +properties: + state: + "$ref": "#/definitions/CtidStatus" +definitions: + CtidStatus: + type: object + properties: + ctid: + description: ctid bookmark + type: string From ec7198ce24e0e92eed05afaf7c2643223e37c8af Mon Sep 17 00:00:00 2001 From: Rodi Reich Zilberman <867491+rodireich@users.noreply.github.com> Date: Sun, 11 Jun 2023 10:19:52 -0700 Subject: [PATCH 02/27] initial ctid for testing --- .../src/main/java/io/airbyte/db/factory/DataSourceFactory.java | 1 + .../integrations/source/postgres/ctid/CtidStateIterator.java | 2 +- .../integrations/source/postgres/ctid/PostgresCtidHandler.java | 3 ++- .../integrations/source/relationaldb/AbstractDbSource.java | 2 +- 4 files changed, 5 insertions(+), 3 deletions(-) diff --git a/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java b/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java index d970b09fbba959..384e8cfa37b31a 100644 --- a/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java +++ b/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java @@ -306,6 +306,7 @@ public DataSource build() { config.setPassword(password); config.setUsername(username); + config.setIdleTimeout(1_000); // TODO: check here /* * Disable to prevent failing on startup. Applications may start prior to the database container * being available. To avoid failing to create the connection pool, disable the fail check. This diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index ff20f271ad4b87..2421744f277296 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -40,7 +40,7 @@ protected AirbyteMessage computeNext() { } if (messageIterator.hasNext()) { - if (count % 10_000 == 0 && StringUtils.isNotBlank(lastCtid)) { + if (count % 1_000_000 == 0 && StringUtils.isNotBlank(lastCtid)) { LOGGER.info("saving ctid state with {}", this.lastCtid); return CtidStateManager.createStateMessage(pair, new CtidStatus().withCtid(lastCtid)); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index f199c3230fe86f..c7bb86e3699831 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -131,6 +131,7 @@ private PreparedStatement createCtidQueryStatement( preparedStatement.setObject(1, "(0,0)"); } LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement); + preparedStatement.setQueryTimeout(120); return preparedStatement; } catch (final SQLException e) { throw new RuntimeException(e); @@ -161,7 +162,7 @@ private AutoCloseableIterator augmentWithLogs(final AutoCloseabl AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()), r -> { final long count = recordCount.incrementAndGet(); - if (count % 10000 == 0) { + if (count % 1_000_000 == 0) { LOGGER.info("Reading stream {}. Records read: {}", streamName, count); } return r; diff --git a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java index b805587ba2d9e1..c315bc64f94f1b 100644 --- a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java +++ b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java @@ -418,7 +418,7 @@ private AutoCloseableIterator createReadIterator(final Database AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()), r -> { final long count = recordCount.incrementAndGet(); - if (count % 10000 == 0) { + if (count % 1_000_000 == 0) { LOGGER.info("Reading stream {}. Records read: {}", streamName, count); } return r; From 1459acb71a9da4a7618707a6ed0111b24301c176 Mon Sep 17 00:00:00 2001 From: Rodi Reich Zilberman <867491+rodireich@users.noreply.github.com> Date: Mon, 12 Jun 2023 22:52:07 -0700 Subject: [PATCH 03/27] initial ctid for testing --- .../db/jdbc/StreamingJdbcDatabase.java | 16 ++++++++++++++ .../source/postgres/PostgresSource.java | 3 ++- .../postgres/ctid/CtidStateIterator.java | 15 +++++++++++-- .../postgres/ctid/CtidStateManager.java | 6 ++++-- .../postgres/ctid/PostgresCtidHandler.java | 21 +++++++++++++++++-- .../internal_models/ctid_internal_models.yaml | 6 ++++++ .../source/relationaldb/AbstractDbSource.java | 2 +- deps.toml | 2 +- 8 files changed, 62 insertions(+), 9 deletions(-) diff --git a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java index 502cce75d737b8..4103591abe251b 100644 --- a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java +++ b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java @@ -4,6 +4,8 @@ package io.airbyte.db.jdbc; +import static java.lang.Thread.sleep; + import com.google.errorprone.annotations.MustBeClosed; import io.airbyte.commons.functional.CheckedFunction; import io.airbyte.db.JdbcCompatibleSourceOperations; @@ -12,6 +14,7 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; +import java.util.Arrays; import java.util.Spliterator; import java.util.Spliterators; import java.util.function.Consumer; @@ -95,13 +98,19 @@ protected Stream toUnsafeStream(final ResultSet resultSet, @Override public boolean tryAdvance(final Consumer action) { try { +// LOGGER.info("tryAdvance"); +// sleep(3000); if (!resultSet.next()) { +// LOGGER.error("exiting tryAdvance"); +// LOGGER.error("stacktrace: \n{}", Arrays.toString(Thread.currentThread().getStackTrace())); resultSet.close(); +// LOGGER.info("return f1"); return false; } final T dataRow = mapper.apply(resultSet); streamingConfig.accept(resultSet, dataRow); action.accept(dataRow); +// LOGGER.info("return t2 {}", dataRow.toString()); return true; } catch (final SQLException e) { LOGGER.error("SQLState: {}, Message: {}", e.getSQLState(), e.getMessage()); @@ -109,6 +118,13 @@ public boolean tryAdvance(final Consumer action) { isStreamFailed = true; // throwing an exception in tryAdvance() method lead to the endless loop in Spliterator and stream // will never close +// LOGGER.info("return f3"); + return false; + } catch (final Exception e) { + LOGGER.error("General exception", e); + streamException = e; + isStreamFailed = true; +// LOGGER.info("return f4"); return false; } } 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 6b0599ef1cef30..58b879cf995c2e 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 @@ -169,7 +169,8 @@ public JsonNode toDatabaseConfig(final JsonNode config) { additionalParameters.forEach(x -> jdbcUrl.append(x).append("&")); jdbcUrl.append(toJDBCQueryParams(sslParameters)); - LOGGER.debug("jdbc url: {}", jdbcUrl.toString()); +// jdbcUrl.append("&options=-c%20statement_timeout=15000"); + LOGGER.info("jdbc url: {}", jdbcUrl.toString()); final ImmutableMap.Builder configBuilder = ImmutableMap.builder() .put(JdbcUtils.USERNAME_KEY, config.get(JdbcUtils.USERNAME_KEY).asText()) .put(JdbcUtils.JDBC_URL_KEY, jdbcUrl.toString()); diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 2421744f277296..3f8d57ab62bdb7 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -1,5 +1,8 @@ package io.airbyte.integrations.source.postgres.ctid; +import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_TYPE; +import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_VERSION; + import com.google.common.collect.AbstractIterator; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; @@ -42,7 +45,11 @@ protected AirbyteMessage computeNext() { if (messageIterator.hasNext()) { if (count % 1_000_000 == 0 && StringUtils.isNotBlank(lastCtid)) { LOGGER.info("saving ctid state with {}", this.lastCtid); - return CtidStateManager.createStateMessage(pair, new CtidStatus().withCtid(lastCtid)); + return CtidStateManager.createStateMessage(pair, + new CtidStatus() + .withVer(CTID_STATUS_VERSION) + .withType(CTID_STATUS_TYPE) + .withCtid(lastCtid)); } // Use try-catch to catch Exception that could occur when connection to the database fails try { @@ -61,7 +68,11 @@ protected AirbyteMessage computeNext() { } } else if (!hasEmittedFinalState) { hasEmittedFinalState = true; - return CtidStateManager.createStateMessage(pair, new CtidStatus().withCtid(lastCtid)); + return CtidStateManager.createStateMessage(pair, + new CtidStatus() + .withVer(CTID_STATUS_VERSION) + .withType(CTID_STATUS_TYPE) + .withCtid(lastCtid)); } else { return endOfData(); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 406a24e80f39ff..26bada47565f6d 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -18,9 +18,9 @@ import org.slf4j.LoggerFactory; public class CtidStateManager { - private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateManager.class); - + public static final long CTID_STATUS_VERSION = 2; + public static final String CTID_STATUS_TYPE = "ctid"; private final Map pairToCtidStatus; private final static AirbyteStateMessage EMPTY_STATE = new AirbyteStateMessage() .withType(AirbyteStateType.STREAM) @@ -41,6 +41,8 @@ private static Map createPairToCtidS final CtidStatus ctidStatus; try { ctidStatus = Jsons.object(stateMessage.getStream().getStreamState(), CtidStatus.class); + assert (ctidStatus.getVer() == CTID_STATUS_VERSION); + assert(ctidStatus.getType().equals(CTID_STATUS_TYPE)); // TODO: check here } catch (final IllegalArgumentException e) { throw new ConfigErrorException("Invalid per-stream state"); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index c7bb86e3699831..eb3abc3b2f0cee 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -3,6 +3,7 @@ import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; import io.airbyte.commons.stream.AirbyteStreamUtils; import io.airbyte.commons.util.AutoCloseableIterator; import io.airbyte.commons.util.AutoCloseableIterators; @@ -82,7 +83,9 @@ public List> getIncrementalIterators( final AutoCloseableIterator queryStream = queryTableCtid(selectedDatabaseFields, table.getNameSpace(), table.getName()); final AutoCloseableIterator recordIterator = getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); final AutoCloseableIterator recordAndMessageIterator = augmentWithState(recordIterator, pair); - iteratorList.add(augmentWithLogs(recordAndMessageIterator, pair, streamName)); + final AutoCloseableIterator logAugmented = augmentWithLogs(recordAndMessageIterator, pair, streamName); + iteratorList.add(swallowCtid(logAugmented, pair)); + } } return iteratorList; @@ -169,8 +172,22 @@ private AutoCloseableIterator augmentWithLogs(final AutoCloseabl }); } - private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, final AirbyteStreamNameNamespacePair pair) { + private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, + final AirbyteStreamNameNamespacePair pair) { return AutoCloseableIterators.transform( autoClosableIterator -> new CtidStateIterator(recordIterator, pair), recordIterator, pair); } + + private AutoCloseableIterator swallowCtid(final AutoCloseableIterator iterator, + final AirbyteStreamNameNamespacePair pair) { + return AutoCloseableIterators.transform(iterator, + pair, + r -> { + if (r.getType() == Type.RECORD) { + ((ObjectNode) r.getRecord().getData()).remove("ctid"); + } + return r; + }); + } + } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml index 3c6cf97b9efaf6..6665cc73a7fce5 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml @@ -10,6 +10,12 @@ definitions: CtidStatus: type: object properties: + ver: + description: version + type: integer + type: + description: state type + type: string ctid: description: ctid bookmark type: string diff --git a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java index c315bc64f94f1b..591058ea48280b 100644 --- a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java +++ b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java @@ -419,7 +419,7 @@ private AutoCloseableIterator createReadIterator(final Database r -> { final long count = recordCount.incrementAndGet(); if (count % 1_000_000 == 0) { - LOGGER.info("Reading stream {}. Records read: {}", streamName, count); + LOGGER.info("Reading stream {}. /airbyte-config/init/: {}", streamName, count); } return r; }); diff --git a/deps.toml b/deps.toml index 42ee78ea9ce40e..c4077a3ded95d3 100644 --- a/deps.toml +++ b/deps.toml @@ -27,7 +27,7 @@ micronaut-jaxrs = "3.4.0" micronaut-security = "3.9.2" micronaut-test = "3.8.0" platform-testcontainers = "1.17.3" -postgresql = "42.3.5" +postgresql = "42.6.0" reactor = "3.5.2" segment = "2.1.1" slf4j = "1.7.36" From 8681c8c5ae32d24e68d5539f5ef59342a1c201c0 Mon Sep 17 00:00:00 2001 From: rodireich Date: Tue, 13 Jun 2023 06:14:01 +0000 Subject: [PATCH 04/27] Automated Commit - Format and Process Resources Changes --- .../db/jdbc/StreamingJdbcDatabase.java | 19 ++++++++----------- 1 file changed, 8 insertions(+), 11 deletions(-) diff --git a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java index 4103591abe251b..481906f41734fc 100644 --- a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java +++ b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java @@ -4,8 +4,6 @@ package io.airbyte.db.jdbc; -import static java.lang.Thread.sleep; - import com.google.errorprone.annotations.MustBeClosed; import io.airbyte.commons.functional.CheckedFunction; import io.airbyte.db.JdbcCompatibleSourceOperations; @@ -14,7 +12,6 @@ import java.sql.PreparedStatement; import java.sql.ResultSet; import java.sql.SQLException; -import java.util.Arrays; import java.util.Spliterator; import java.util.Spliterators; import java.util.function.Consumer; @@ -98,19 +95,19 @@ protected Stream toUnsafeStream(final ResultSet resultSet, @Override public boolean tryAdvance(final Consumer action) { try { -// LOGGER.info("tryAdvance"); -// sleep(3000); + // LOGGER.info("tryAdvance"); + // sleep(3000); if (!resultSet.next()) { -// LOGGER.error("exiting tryAdvance"); -// LOGGER.error("stacktrace: \n{}", Arrays.toString(Thread.currentThread().getStackTrace())); + // LOGGER.error("exiting tryAdvance"); + // LOGGER.error("stacktrace: \n{}", Arrays.toString(Thread.currentThread().getStackTrace())); resultSet.close(); -// LOGGER.info("return f1"); + // LOGGER.info("return f1"); return false; } final T dataRow = mapper.apply(resultSet); streamingConfig.accept(resultSet, dataRow); action.accept(dataRow); -// LOGGER.info("return t2 {}", dataRow.toString()); + // LOGGER.info("return t2 {}", dataRow.toString()); return true; } catch (final SQLException e) { LOGGER.error("SQLState: {}, Message: {}", e.getSQLState(), e.getMessage()); @@ -118,13 +115,13 @@ public boolean tryAdvance(final Consumer action) { isStreamFailed = true; // throwing an exception in tryAdvance() method lead to the endless loop in Spliterator and stream // will never close -// LOGGER.info("return f3"); + // LOGGER.info("return f3"); return false; } catch (final Exception e) { LOGGER.error("General exception", e); streamException = e; isStreamFailed = true; -// LOGGER.info("return f4"); + // LOGGER.info("return f4"); return false; } } From 14371ea0c087523120f7c9b8fe07546e3dbead8c Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 13 Jun 2023 15:47:12 +0530 Subject: [PATCH 05/27] add version and state type to xmin status --- .../integrations/source/postgres/PostgresQueryUtils.java | 5 ++++- .../main/resources/internal_models/internal_models.yaml | 8 ++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 7043037bd70fdc..cd25fc3121caaf 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -11,6 +11,7 @@ import io.airbyte.db.jdbc.JdbcDatabase; import io.airbyte.db.jdbc.JdbcUtils; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; +import io.airbyte.integrations.source.postgres.internal.models.XminStatus.StateType; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; import java.sql.SQLException; import java.util.List; @@ -89,7 +90,9 @@ public static XminStatus getXminStatus(final JdbcDatabase database) throws SQLEx return new XminStatus() .withNumWraparound(result.get(NUM_WRAPAROUND_COL).asLong()) .withXminXidValue(result.get(XMIN_XID_VALUE_COL).asLong()) - .withXminRawValue(result.get(XMIN_RAW_VALUE_COL).asLong()); + .withXminRawValue(result.get(XMIN_RAW_VALUE_COL).asLong()) + .withVersion(2L) + .withStateType(StateType.XMIN); } public static void logFullVacuumStatus(final JdbcDatabase database, final ConfiguredAirbyteCatalog catalog, final String quoteString) { diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml index 8455adb4dead7d..34efa326214eed 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml @@ -10,6 +10,14 @@ definitions: XminStatus: type: object properties: + version: + description: Version of state. + type: integer + state_type: + description: Enum to define the sync mode of state. + type: string + enum: + - xmin num_wraparound: description: Number of times the Xmin value has wrapped around. type: integer From eb7255c2ff287d5049d95b9c65d6252849a6d12f Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 13 Jun 2023 17:24:59 +0530 Subject: [PATCH 06/27] add logic to swtich between xmin and ctid sync --- .../source/postgres/PostgresSource.java | 82 +++++++++++++++++-- .../postgres/ctid/CtidStateIterator.java | 26 ++++-- .../postgres/ctid/PostgresCtidHandler.java | 23 ++++-- .../postgres/xmin/XminStateManager.java | 13 ++- .../internal_models/ctid_internal_models.yaml | 4 + 5 files changed, 125 insertions(+), 23 deletions(-) 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 58b879cf995c2e..32b8df3565e44c 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 @@ -27,10 +27,12 @@ import static java.util.stream.Collectors.toSet; import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.collect.Sets; import datadog.trace.api.Trace; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.commons.features.EnvVariableFeatureFlags; @@ -70,11 +72,13 @@ import io.airbyte.protocol.models.v0.AirbyteConnectionStatus.Status; import io.airbyte.protocol.models.v0.AirbyteEstimateTraceMessage.Type; import io.airbyte.protocol.models.v0.AirbyteMessage; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.v0.AirbyteStream; import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.StreamDescriptor; import java.net.URI; import java.net.URISyntaxException; import java.nio.file.Path; @@ -85,7 +89,9 @@ import java.time.Duration; import java.time.Instant; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -94,6 +100,7 @@ import java.util.Set; import java.util.function.Supplier; import java.util.stream.Collectors; +import java.util.stream.Stream; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -455,17 +462,78 @@ public List> getIncrementalIterators(final AutoCloseableIterators.lazyIterator(incrementalIteratorSupplier, null))); } else if (PostgresUtils.isXmin(sourceConfig) && isIncrementalSyncMode(catalog)) { - final XminStateManager xminStateManager = new XminStateManager(stateManager.getRawStateMessages()); - final PostgresXminHandler handler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); - return handler.getIncrementalIterators(catalog, tableNameToTable, emittedAt); + final List rawStateMessages = stateManager.getRawStateMessages(); + + final List statesFromCtidSync = new ArrayList<>(); + final List statesFromXminSync = new ArrayList<>(); + + final Set alreadySeenStreams = new HashSet<>(); + final Set streamsStillInCtidSync = new HashSet<>(); + rawStateMessages.forEach(s -> { + final JsonNode streamState = s.getStream().getStreamState(); + final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); + final AirbyteStateMessage clonedState = Jsons.clone(s); + if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { + statesFromCtidSync.add(clonedState); + streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + } else { + statesFromXminSync.add(clonedState); + } + alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + }); + + final List newlyAddedStreams = identifyNewlyAddedStreams(catalog, alreadySeenStreams); + + final List configuredStreamsStillInCtidSync = catalog.getStreams().stream() + .filter(stream -> streamsStillInCtidSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) + .map(Jsons::clone) + .toList(); + + configuredStreamsStillInCtidSync.addAll(newlyAddedStreams); + + final List streamsInXminSync = catalog.getStreams().stream() + .filter(stream -> !configuredStreamsStillInCtidSync.contains(stream)) + .map(Jsons::clone) + .toList(); + + final XminStateManager xminStateManager = new XminStateManager(statesFromXminSync); + final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); + + final List> xminIterator = xminHandler.getIncrementalIterators( + new ConfiguredAirbyteCatalog().withStreams(streamsInXminSync), tableNameToTable, emittedAt); + + if (configuredStreamsStillInCtidSync.isEmpty()) { + return xminIterator; + } + + final CtidStateManager ctidStateManager = new CtidStateManager(statesFromCtidSync); + final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), ctidStateManager, + x -> new ObjectMapper().valueToTree(xminStatus), + (pair, jsonState) -> XminStateManager.getAirbyteStateMessage(pair, Jsons.object(jsonState, XminStatus.class))); + final List> ctidIterator = ctidHandler.getIncrementalIterators( + new ConfiguredAirbyteCatalog().withStreams(configuredStreamsStillInCtidSync), tableNameToTable, emittedAt); + return Stream + .of(ctidIterator, xminIterator) + .flatMap(Collection::stream) + .collect(Collectors.toList()); } else { - final CtidStateManager ctidStateManager = new CtidStateManager(stateManager.getRawStateMessages()); - final PostgresCtidHandler handler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), null, ctidStateManager); - return handler.getIncrementalIterators(catalog, tableNameToTable, emittedAt); -// return super.getIncrementalIterators(database, catalog, tableNameToTable, stateManager, emittedAt); + return super.getIncrementalIterators(database, catalog, tableNameToTable, stateManager, emittedAt); } } + protected List identifyNewlyAddedStreams(final ConfiguredAirbyteCatalog catalog, final Set alreadySeenStreams) { + final Set allStreams = AirbyteStreamNameNamespacePair.fromConfiguredCatalog(catalog); + + final Set newlyAddedStreams = new HashSet<>(Sets.difference(allStreams, alreadySeenStreams)); + + return catalog.getStreams().stream() + .filter(stream -> newlyAddedStreams.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) + .map(Jsons::clone) + .collect(Collectors.toList()); + } + + + @Override public Set getPrivilegesTableForCurrentUser(final JdbcDatabase database, final String schema) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 3f8d57ab62bdb7..e6e1df691b663e 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -3,12 +3,18 @@ import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_TYPE; import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_VERSION; +import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.AbstractIterator; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.v0.AirbyteMessage; +import io.airbyte.protocol.models.v0.AirbyteMessage.Type; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; import java.util.Iterator; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; import javax.annotation.CheckForNull; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -22,11 +28,18 @@ public class CtidStateIterator extends AbstractIterator implemen private boolean hasEmittedFinalState; private boolean hasCaughtException = false; private String lastCtid; + private final JsonNode streamStateForIncrementalRun; + final BiFunction finalStateMessageSupplier; final AtomicLong recordCount = new AtomicLong(); + public CtidStateIterator(final Iterator messageIterator, - final AirbyteStreamNameNamespacePair pair) { + final AirbyteStreamNameNamespacePair pair, + final JsonNode streamStateForIncrementalRun, + final BiFunction finalStateMessageSupplier) { this.messageIterator = messageIterator; this.pair = pair; + this.streamStateForIncrementalRun = streamStateForIncrementalRun; + this.finalStateMessageSupplier = finalStateMessageSupplier; } @CheckForNull @@ -49,7 +62,8 @@ protected AirbyteMessage computeNext() { new CtidStatus() .withVer(CTID_STATUS_VERSION) .withType(CTID_STATUS_TYPE) - .withCtid(lastCtid)); + .withCtid(lastCtid) + .withIncrementalState(streamStateForIncrementalRun)); } // Use try-catch to catch Exception that could occur when connection to the database fails try { @@ -68,11 +82,9 @@ protected AirbyteMessage computeNext() { } } else if (!hasEmittedFinalState) { hasEmittedFinalState = true; - return CtidStateManager.createStateMessage(pair, - new CtidStatus() - .withVer(CTID_STATUS_VERSION) - .withType(CTID_STATUS_TYPE) - .withCtid(lastCtid)); + return new AirbyteMessage() + .withType(Type.STATE) + .withState(finalStateMessageSupplier.apply(pair, streamStateForIncrementalRun)); } else { return endOfData(); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index eb3abc3b2f0cee..27a104cd262a61 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -19,6 +19,7 @@ import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteMessage.Type; import io.airbyte.protocol.models.v0.AirbyteRecordMessage; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.v0.AirbyteStream; import io.airbyte.protocol.models.v0.CatalogHelpers; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; @@ -32,6 +33,9 @@ import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicLong; +import java.util.function.BiFunction; +import java.util.function.Function; +import java.util.function.Supplier; import java.util.stream.Stream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -42,18 +46,22 @@ public class PostgresCtidHandler { private final JdbcDatabase database; private final JdbcCompatibleSourceOperations sourceOperations; private final String quoteString; - private final CtidStatus ctidStatus; private final CtidStateManager ctidStateManager; + private final Function streamStateForIncrementalRunSupplier; + private final BiFunction finalStateMessageSupplier; + public PostgresCtidHandler(final JdbcDatabase database, final JdbcCompatibleSourceOperations sourceOperations, final String quoteString, - final CtidStatus ctidStatus, - final CtidStateManager ctidStateManager) { + final CtidStateManager ctidStateManager, + final Function streamStateForIncrementalRunSupplier, + final BiFunction finalStateMessageSupplier) { this.database = database; this.sourceOperations = sourceOperations; this.quoteString = quoteString; - this.ctidStatus = ctidStatus; this.ctidStateManager = ctidStateManager; + this.streamStateForIncrementalRunSupplier = streamStateForIncrementalRunSupplier; + this.finalStateMessageSupplier = finalStateMessageSupplier; } public List> getIncrementalIterators( final ConfiguredAirbyteCatalog catalog, @@ -174,8 +182,13 @@ private AutoCloseableIterator augmentWithLogs(final AutoCloseabl private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, final AirbyteStreamNameNamespacePair pair) { + + final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(pair); + final JsonNode incrementalState = currentCtidStatus.getIncrementalState() != null ? currentCtidStatus.getIncrementalState() : streamStateForIncrementalRunSupplier.apply(pair); + + return AutoCloseableIterators.transform( - autoClosableIterator -> new CtidStateIterator(recordIterator, pair), recordIterator, pair); + autoClosableIterator -> new CtidStateIterator(recordIterator, pair, incrementalState, finalStateMessageSupplier ), recordIterator, pair); } private AutoCloseableIterator swallowCtid(final AutoCloseableIterator iterator, diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java index 5459027251a6c3..d2cd16a39ef367 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java @@ -73,6 +73,14 @@ public XminStatus getXminStatus(final AirbyteStreamNameNamespacePair pair) { * @return AirbyteMessage which includes information on state of records read so far */ public static AirbyteMessage createStateMessage(final AirbyteStreamNameNamespacePair pair, final XminStatus xminStatus) { + final AirbyteStateMessage stateMessage = getAirbyteStateMessage(pair, xminStatus); + + return new AirbyteMessage() + .withType(Type.STATE) + .withState(stateMessage); + } + + public static AirbyteStateMessage getAirbyteStateMessage(final AirbyteStreamNameNamespacePair pair, final XminStatus xminStatus) { final AirbyteStreamState airbyteStreamState = new AirbyteStreamState() .withStreamDescriptor( @@ -86,10 +94,7 @@ public static AirbyteMessage createStateMessage(final AirbyteStreamNameNamespace new AirbyteStateMessage() .withType(AirbyteStateType.STREAM) .withStream(airbyteStreamState); - - return new AirbyteMessage() - .withType(Type.STATE) - .withState(stateMessage); + return stateMessage; } } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml index 6665cc73a7fce5..f866928bca8b70 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml @@ -19,3 +19,7 @@ definitions: ctid: description: ctid bookmark type: string + incremental_state: + description: "State to switch to after completion of ctid snapsho" + type: object + existingJavaType: com.fasterxml.jackson.databind.JsonNode From 2215f64fc722de248499cbaa4e7002630619e547 Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 13 Jun 2023 21:37:28 +0530 Subject: [PATCH 07/27] npe fixes --- .../source/postgres/PostgresSource.java | 73 +++++++++++-------- .../postgres/ctid/PostgresCtidHandler.java | 4 +- 2 files changed, 46 insertions(+), 31 deletions(-) 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 32b8df3565e44c..bf511ba81b641a 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 @@ -469,49 +469,62 @@ public List> getIncrementalIterators(final final Set alreadySeenStreams = new HashSet<>(); final Set streamsStillInCtidSync = new HashSet<>(); - rawStateMessages.forEach(s -> { - final JsonNode streamState = s.getStream().getStreamState(); - final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); - final AirbyteStateMessage clonedState = Jsons.clone(s); - if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { - statesFromCtidSync.add(clonedState); - streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); - } else { - statesFromXminSync.add(clonedState); - } - alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); - }); - final List newlyAddedStreams = identifyNewlyAddedStreams(catalog, alreadySeenStreams); + if (rawStateMessages != null) { + rawStateMessages.forEach(s -> { + final JsonNode streamState = s.getStream().getStreamState(); + final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); + if (streamState == null || streamDescriptor == null) { + return; + } + final AirbyteStateMessage clonedState = Jsons.clone(s); + if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { + statesFromCtidSync.add(clonedState); + streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + } else { + statesFromXminSync.add(clonedState); + } + alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + }); + } - final List configuredStreamsStillInCtidSync = catalog.getStreams().stream() + final List newlyAddedStreams = identifyNewlyAddedStreams(catalog, alreadySeenStreams); + final List streamsForCtidSync = new ArrayList<>(); + catalog.getStreams().stream() .filter(stream -> streamsStillInCtidSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) .map(Jsons::clone) - .toList(); + .forEach(streamsForCtidSync::add); - configuredStreamsStillInCtidSync.addAll(newlyAddedStreams); + streamsForCtidSync.addAll(newlyAddedStreams); - final List streamsInXminSync = catalog.getStreams().stream() - .filter(stream -> !configuredStreamsStillInCtidSync.contains(stream)) + final List streamsForXminSync = catalog.getStreams().stream() + .filter(stream -> !streamsForCtidSync.contains(stream)) .map(Jsons::clone) .toList(); - final XminStateManager xminStateManager = new XminStateManager(statesFromXminSync); - final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); + LOGGER.info("Streams to be synced via ctid : {}", streamsForCtidSync.size()); + LOGGER.info("Streams to be synced via xmin : {}", streamsForXminSync.size()); + + final List> ctidIterator = new ArrayList<>(); + final List> xminIterator = new ArrayList<>(); + + if (!streamsForCtidSync.isEmpty()) { + final CtidStateManager ctidStateManager = new CtidStateManager(statesFromCtidSync); + final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), ctidStateManager, + x -> new ObjectMapper().valueToTree(xminStatus), + (pair, jsonState) -> XminStateManager.getAirbyteStateMessage(pair, Jsons.object(jsonState, XminStatus.class))); + ctidIterator.addAll(ctidHandler.getIncrementalIterators( + new ConfiguredAirbyteCatalog().withStreams(streamsForCtidSync), tableNameToTable, emittedAt)); + } - final List> xminIterator = xminHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsInXminSync), tableNameToTable, emittedAt); + if (!streamsForXminSync.isEmpty()) { + final XminStateManager xminStateManager = new XminStateManager(statesFromXminSync); + final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); - if (configuredStreamsStillInCtidSync.isEmpty()) { - return xminIterator; + xminIterator.addAll(xminHandler.getIncrementalIterators( + new ConfiguredAirbyteCatalog().withStreams(streamsForXminSync), tableNameToTable, emittedAt)); } - final CtidStateManager ctidStateManager = new CtidStateManager(statesFromCtidSync); - final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), ctidStateManager, - x -> new ObjectMapper().valueToTree(xminStatus), - (pair, jsonState) -> XminStateManager.getAirbyteStateMessage(pair, Jsons.object(jsonState, XminStatus.class))); - final List> ctidIterator = ctidHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(configuredStreamsStillInCtidSync), tableNameToTable, emittedAt); return Stream .of(ctidIterator, xminIterator) .flatMap(Collection::stream) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index 27a104cd262a61..e371e638285129 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -184,7 +184,9 @@ private AutoCloseableIterator augmentWithState(final AutoCloseab final AirbyteStreamNameNamespacePair pair) { final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(pair); - final JsonNode incrementalState = currentCtidStatus.getIncrementalState() != null ? currentCtidStatus.getIncrementalState() : streamStateForIncrementalRunSupplier.apply(pair); + final JsonNode incrementalState = + (currentCtidStatus == null || currentCtidStatus.getIncrementalState() == null) ? streamStateForIncrementalRunSupplier.apply(pair) + : currentCtidStatus.getIncrementalState(); return AutoCloseableIterators.transform( From 52d09a06c14c1eadfe5e8985ffe94fd4d833c7bc Mon Sep 17 00:00:00 2001 From: subodh Date: Wed, 14 Jun 2023 00:08:21 +0530 Subject: [PATCH 08/27] use enum --- .../integrations/source/postgres/ctid/CtidStateIterator.java | 5 +---- .../integrations/source/postgres/ctid/CtidStateManager.java | 3 +-- .../main/resources/internal_models/ctid_internal_models.yaml | 4 +++- 3 files changed, 5 insertions(+), 7 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index e6e1df691b663e..d3dd9d7d3b65bb 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -1,6 +1,5 @@ package io.airbyte.integrations.source.postgres.ctid; -import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_TYPE; import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_VERSION; import com.fasterxml.jackson.databind.JsonNode; @@ -13,8 +12,6 @@ import java.util.Iterator; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; -import java.util.function.Function; -import java.util.function.Supplier; import javax.annotation.CheckForNull; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; @@ -61,7 +58,7 @@ protected AirbyteMessage computeNext() { return CtidStateManager.createStateMessage(pair, new CtidStatus() .withVer(CTID_STATUS_VERSION) - .withType(CTID_STATUS_TYPE) + .withType(CtidStatus.Type.CTID) .withCtid(lastCtid) .withIncrementalState(streamStateForIncrementalRun)); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 26bada47565f6d..296d7a65c1dacd 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -20,7 +20,6 @@ public class CtidStateManager { private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateManager.class); public static final long CTID_STATUS_VERSION = 2; - public static final String CTID_STATUS_TYPE = "ctid"; private final Map pairToCtidStatus; private final static AirbyteStateMessage EMPTY_STATE = new AirbyteStateMessage() .withType(AirbyteStateType.STREAM) @@ -42,7 +41,7 @@ private static Map createPairToCtidS try { ctidStatus = Jsons.object(stateMessage.getStream().getStreamState(), CtidStatus.class); assert (ctidStatus.getVer() == CTID_STATUS_VERSION); - assert(ctidStatus.getType().equals(CTID_STATUS_TYPE)); // TODO: check here + assert(ctidStatus.getType().equals(CtidStatus.Type.CTID)); // TODO: check here } catch (final IllegalArgumentException e) { throw new ConfigErrorException("Invalid per-stream state"); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml index f866928bca8b70..8a3968e334cb99 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml @@ -16,10 +16,12 @@ definitions: type: description: state type type: string + enum: + - ctid ctid: description: ctid bookmark type: string incremental_state: - description: "State to switch to after completion of ctid snapsho" + description: "State to switch to after completion of ctid initial sync" type: object existingJavaType: com.fasterxml.jackson.databind.JsonNode From c6bebb5d077e456b67d26124e9e70eac02be01e5 Mon Sep 17 00:00:00 2001 From: subodh Date: Thu, 15 Jun 2023 14:52:59 +0530 Subject: [PATCH 09/27] refactor --- .../source/postgres/PostgresQueryUtils.java | 2 +- .../source/postgres/PostgresSource.java | 72 ++---------- .../postgres/ctid/CtidStateIterator.java | 8 +- .../postgres/ctid/CtidStateManager.java | 8 +- .../source/postgres/xmin/XminCtidUtils.java | 111 ++++++++++++++++++ .../internal_models/ctid_internal_models.yaml | 27 ----- .../internal_models/internal_models.yaml | 34 +++++- 7 files changed, 162 insertions(+), 100 deletions(-) create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java delete mode 100644 airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index cd25fc3121caaf..232d9c1ff155b1 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -10,8 +10,8 @@ import com.google.common.base.Preconditions; import io.airbyte.db.jdbc.JdbcDatabase; import io.airbyte.db.jdbc.JdbcUtils; +import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; -import io.airbyte.integrations.source.postgres.internal.models.XminStatus.StateType; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; import java.sql.SQLException; import java.util.List; 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 bf511ba81b641a..1308413f39c6fb 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 @@ -21,6 +21,7 @@ import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TABLE_ESTIMATE_QUERY; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TOTAL_BYTES_RESULT_COL; import static io.airbyte.integrations.source.postgres.PostgresUtils.isIncrementalSyncMode; +import static io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.categoriseStreams; import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; import static io.airbyte.integrations.util.PostgresSslConnectionUtils.PARAM_SSL_MODE; import static java.util.stream.Collectors.toList; @@ -62,6 +63,7 @@ import io.airbyte.integrations.source.postgres.ctid.PostgresCtidHandler; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; import io.airbyte.integrations.source.postgres.xmin.PostgresXminHandler; +import io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.StreamsCategorised; import io.airbyte.integrations.source.postgres.xmin.XminStateManager; import io.airbyte.integrations.source.relationaldb.TableInfo; import io.airbyte.integrations.source.relationaldb.state.StateManager; @@ -462,67 +464,28 @@ public List> getIncrementalIterators(final AutoCloseableIterators.lazyIterator(incrementalIteratorSupplier, null))); } else if (PostgresUtils.isXmin(sourceConfig) && isIncrementalSyncMode(catalog)) { - final List rawStateMessages = stateManager.getRawStateMessages(); - - final List statesFromCtidSync = new ArrayList<>(); - final List statesFromXminSync = new ArrayList<>(); - - final Set alreadySeenStreams = new HashSet<>(); - final Set streamsStillInCtidSync = new HashSet<>(); - - if (rawStateMessages != null) { - rawStateMessages.forEach(s -> { - final JsonNode streamState = s.getStream().getStreamState(); - final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); - if (streamState == null || streamDescriptor == null) { - return; - } - final AirbyteStateMessage clonedState = Jsons.clone(s); - if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { - statesFromCtidSync.add(clonedState); - streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); - } else { - statesFromXminSync.add(clonedState); - } - alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); - }); - } - - final List newlyAddedStreams = identifyNewlyAddedStreams(catalog, alreadySeenStreams); - final List streamsForCtidSync = new ArrayList<>(); - catalog.getStreams().stream() - .filter(stream -> streamsStillInCtidSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) - .map(Jsons::clone) - .forEach(streamsForCtidSync::add); - - streamsForCtidSync.addAll(newlyAddedStreams); - - final List streamsForXminSync = catalog.getStreams().stream() - .filter(stream -> !streamsForCtidSync.contains(stream)) - .map(Jsons::clone) - .toList(); - - LOGGER.info("Streams to be synced via ctid : {}", streamsForCtidSync.size()); - LOGGER.info("Streams to be synced via xmin : {}", streamsForXminSync.size()); + final StreamsCategorised streamsCategorised = categoriseStreams(stateManager, catalog); + LOGGER.info("Streams to be synced via ctid : {}", streamsCategorised.ctidStreams.streamsForCtidSync.size()); + LOGGER.info("Streams to be synced via xmin : {}", streamsCategorised.xminStreams.streamsForXminSync.size()); final List> ctidIterator = new ArrayList<>(); final List> xminIterator = new ArrayList<>(); - if (!streamsForCtidSync.isEmpty()) { - final CtidStateManager ctidStateManager = new CtidStateManager(statesFromCtidSync); + if (!streamsCategorised.ctidStreams.streamsForCtidSync.isEmpty()) { + final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams.statesFromCtidSync); final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), ctidStateManager, x -> new ObjectMapper().valueToTree(xminStatus), (pair, jsonState) -> XminStateManager.getAirbyteStateMessage(pair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsForCtidSync), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams.streamsForCtidSync), tableNameToTable, emittedAt)); } - if (!streamsForXminSync.isEmpty()) { - final XminStateManager xminStateManager = new XminStateManager(statesFromXminSync); + if (!streamsCategorised.xminStreams.streamsForXminSync.isEmpty()) { + final XminStateManager xminStateManager = new XminStateManager(streamsCategorised.xminStreams.statesFromXminSync); final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); xminIterator.addAll(xminHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsForXminSync), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.xminStreams.streamsForXminSync), tableNameToTable, emittedAt)); } return Stream @@ -534,19 +497,6 @@ public List> getIncrementalIterators(final } } - protected List identifyNewlyAddedStreams(final ConfiguredAirbyteCatalog catalog, final Set alreadySeenStreams) { - final Set allStreams = AirbyteStreamNameNamespacePair.fromConfiguredCatalog(catalog); - - final Set newlyAddedStreams = new HashSet<>(Sets.difference(allStreams, alreadySeenStreams)); - - return catalog.getStreams().stream() - .filter(stream -> newlyAddedStreams.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) - .map(Jsons::clone) - .collect(Collectors.toList()); - } - - - @Override public Set getPrivilegesTableForCurrentUser(final JdbcDatabase database, final String schema) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index d3dd9d7d3b65bb..2623ebf65c300e 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -5,6 +5,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.AbstractIterator; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteMessage.Type; @@ -55,10 +56,11 @@ protected AirbyteMessage computeNext() { if (messageIterator.hasNext()) { if (count % 1_000_000 == 0 && StringUtils.isNotBlank(lastCtid)) { LOGGER.info("saving ctid state with {}", this.lastCtid); - return CtidStateManager.createStateMessage(pair, + //TODO (Rodi): To add relation_filenode attribute in the CtidStatus + return CtidStateManager.createPerStreamStateMessage(pair, new CtidStatus() - .withVer(CTID_STATUS_VERSION) - .withType(CtidStatus.Type.CTID) + .withVersion(CTID_STATUS_VERSION) + .withStateType(StateType.CTID) .withCtid(lastCtid) .withIncrementalState(streamStateForIncrementalRun)); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 296d7a65c1dacd..0cfc642f0bf8e8 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -3,6 +3,7 @@ import com.fasterxml.jackson.databind.ObjectMapper; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.Jsons; import io.airbyte.protocol.models.v0.AirbyteStateMessage; @@ -40,8 +41,8 @@ private static Map createPairToCtidS final CtidStatus ctidStatus; try { ctidStatus = Jsons.object(stateMessage.getStream().getStreamState(), CtidStatus.class); - assert (ctidStatus.getVer() == CTID_STATUS_VERSION); - assert(ctidStatus.getType().equals(CtidStatus.Type.CTID)); // TODO: check here + assert (ctidStatus.getVersion() == CTID_STATUS_VERSION); + assert(ctidStatus.getStateType().equals(StateType.CTID)); // TODO: check here } catch (final IllegalArgumentException e) { throw new ConfigErrorException("Invalid per-stream state"); } @@ -56,7 +57,8 @@ public CtidStatus getCtidStatus(final AirbyteStreamNameNamespacePair pair) { return pairToCtidStatus.get(pair); } - public static AirbyteMessage createStateMessage(final AirbyteStreamNameNamespacePair pair, final CtidStatus ctidStatus) { + //TODO : We will need a similar method to generate a GLOBAL state message for CDC + public static AirbyteMessage createPerStreamStateMessage(final AirbyteStreamNameNamespacePair pair, final CtidStatus ctidStatus) { final AirbyteStreamState airbyteStreamState = new AirbyteStreamState() .withStreamDescriptor( diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java new file mode 100644 index 00000000000000..6e78378c259dbb --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -0,0 +1,111 @@ +package io.airbyte.integrations.source.postgres.xmin; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.Sets; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.relationaldb.state.StateManager; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; +import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.StreamDescriptor; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.stream.Collectors; + +public class XminCtidUtils { + + public static StreamsCategorised categoriseStreams(final StateManager stateManager, final ConfiguredAirbyteCatalog fullCatalog) { + final List rawStateMessages = stateManager.getRawStateMessages(); + final List statesFromCtidSync = new ArrayList<>(); + final List statesFromXminSync = new ArrayList<>(); + + final Set alreadySeenStreams = new HashSet<>(); + final Set streamsStillInCtidSync = new HashSet<>(); + + if (rawStateMessages != null) { + rawStateMessages.forEach(s -> { + final JsonNode streamState = s.getStream().getStreamState(); + final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); + if (streamState == null || streamDescriptor == null) { + return; + } + final AirbyteStateMessage clonedState = Jsons.clone(s); + if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { + statesFromCtidSync.add(clonedState); + streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + } else { + statesFromXminSync.add(clonedState); + } + alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + }); + } + + final List newlyAddedStreams = identifyNewlyAddedStreams(fullCatalog, alreadySeenStreams); + final List streamsForCtidSync = new ArrayList<>(); + fullCatalog.getStreams().stream() + .filter(stream -> streamsStillInCtidSync.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) + .map(Jsons::clone) + .forEach(streamsForCtidSync::add); + + streamsForCtidSync.addAll(newlyAddedStreams); + + final List streamsForXminSync = fullCatalog.getStreams().stream() + .filter(stream -> !streamsForCtidSync.contains(stream)) + .map(Jsons::clone) + .toList(); + + return new StreamsCategorised(new CtidStreams(streamsForCtidSync, statesFromCtidSync), new XminStreams(streamsForXminSync, statesFromXminSync)); + } + + private static List identifyNewlyAddedStreams(final ConfiguredAirbyteCatalog fullCatalog, + final Set alreadySeenStreams) { + final Set allStreams = AirbyteStreamNameNamespacePair.fromConfiguredCatalog(fullCatalog); + + final Set newlyAddedStreams = new HashSet<>(Sets.difference(allStreams, alreadySeenStreams)); + + return fullCatalog.getStreams().stream() + .filter(stream -> newlyAddedStreams.contains(AirbyteStreamNameNamespacePair.fromAirbyteStream(stream.getStream()))) + .map(Jsons::clone) + .collect(Collectors.toList()); + } + + + public static class StreamsCategorised { + + public final CtidStreams ctidStreams; + public final XminStreams xminStreams; + + public StreamsCategorised(final CtidStreams ctidStreams, final XminStreams xminStreams) { + this.ctidStreams = ctidStreams; + this.xminStreams = xminStreams; + } + } + + public static class CtidStreams { + + public final List streamsForCtidSync; + public final List statesFromCtidSync; + + public CtidStreams(final List streamsForCtidSync, + final List statesFromCtidSync) { + this.streamsForCtidSync = streamsForCtidSync; + this.statesFromCtidSync = statesFromCtidSync; + } + } + + public static class XminStreams { + + public final List streamsForXminSync; + public final List statesFromXminSync; + + public XminStreams(final List streamsForXminSync, + final List statesFromXminSync) { + this.streamsForXminSync = streamsForXminSync; + this.statesFromXminSync = statesFromXminSync; + } + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml deleted file mode 100644 index 8a3968e334cb99..00000000000000 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/ctid_internal_models.yaml +++ /dev/null @@ -1,27 +0,0 @@ ---- -"$schema": http://json-schema.org/draft-07/schema# -title: Postgres Models -type: object -description: Postgres Models -properties: - state: - "$ref": "#/definitions/CtidStatus" -definitions: - CtidStatus: - type: object - properties: - ver: - description: version - type: integer - type: - description: state type - type: string - enum: - - ctid - ctid: - description: ctid bookmark - type: string - incremental_state: - description: "State to switch to after completion of ctid initial sync" - type: object - existingJavaType: com.fasterxml.jackson.databind.JsonNode diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml index 34efa326214eed..cd43c95bf60ead 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml @@ -4,9 +4,19 @@ title: Postgres Models type: object description: Postgres Models properties: - state: + state_type: + "$ref": "#/definitions/StateType" + xmin_state: "$ref": "#/definitions/XminStatus" + ctid_state: + "$ref": "#/definitions/CtidStatus" definitions: + StateType: + description: Enum to define the sync mode of state. + type: string + enum: + - xmin + - ctid XminStatus: type: object properties: @@ -14,10 +24,7 @@ definitions: description: Version of state. type: integer state_type: - description: Enum to define the sync mode of state. - type: string - enum: - - xmin + "$ref": "#/definitions/StateType" num_wraparound: description: Number of times the Xmin value has wrapped around. type: integer @@ -27,3 +34,20 @@ definitions: xmin_raw_value: description: The raw value of the xmin snapshot. If no wraparound has occurred, this should be the same as 2. type: integer + CtidStatus: + type: object + properties: + version: + description: Version of state. + type: integer + state_type: + "$ref": "#/definitions/StateType" + ctid: + description: ctid bookmark + type: string + incremental_state: + description: State to switch to after completion of ctid initial sync + type: object + existingJavaType: com.fasterxml.jackson.databind.JsonNode + relation_filenode: + type: integer \ No newline at end of file From 286c5cd68ae20ae3c837910d76568bec4a32b5b0 Mon Sep 17 00:00:00 2001 From: subodh Date: Fri, 16 Jun 2023 14:31:29 +0530 Subject: [PATCH 10/27] add relation node logic + validation for vacuuming + more refactor --- .../source/postgres/PostgresQueryUtils.java | 61 +++++++++++++------ .../source/postgres/PostgresSource.java | 22 ++++--- .../postgres/ctid/CtidStateIterator.java | 57 ++++++++++------- .../postgres/ctid/CtidStateManager.java | 22 +++++-- .../postgres/ctid/PostgresCtidHandler.java | 21 ++++++- .../source/postgres/xmin/XminCtidUtils.java | 2 +- 6 files changed, 128 insertions(+), 57 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 232d9c1ff155b1..83e36eee835588 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -12,9 +12,13 @@ import io.airbyte.db.jdbc.JdbcUtils; import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; -import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; import java.util.List; +import java.util.Map; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -95,33 +99,56 @@ public static XminStatus getXminStatus(final JdbcDatabase database) throws SQLEx .withStateType(StateType.XMIN); } - public static void logFullVacuumStatus(final JdbcDatabase database, final ConfiguredAirbyteCatalog catalog, final String quoteString) { - catalog.getStreams().forEach(stream -> { + public static Map fileNode(final JdbcDatabase database, final List streams, final String quoteString) { + final Map fileNodes = new HashMap<>(); + streams.forEach(stream -> { + final AirbyteStreamNameNamespacePair namespacePair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()); + final long l = fileNode(database, namespacePair, quoteString); + fileNodes.put(namespacePair, l); + }); + return fileNodes; + } + + public static long fileNode(final JdbcDatabase database, final AirbyteStreamNameNamespacePair stream, final String quoteString) { + try { + final String streamName = stream.getName(); + final String schemaName = stream.getNamespace(); + final String fullTableName = + getFullyQualifiedTableNameWithQuoting(schemaName, streamName, quoteString); + final List jsonNodes = database.bufferedResultSetQuery( + conn -> conn.prepareStatement(CTID_FULL_VACUUM_REL_FILENODE_QUERY.formatted(fullTableName)).executeQuery(), + resultSet -> JdbcUtils.getDefaultSourceOperations().rowToJson(resultSet)); + Preconditions.checkState(jsonNodes.size() == 1); + final long relationFilenode = jsonNodes.get(0).get("pg_relation_filenode").asLong(); + LOGGER.info("Relation filenode is for stream {} is {}", fullTableName, relationFilenode); + return relationFilenode; + } catch (SQLException e) { + throw new RuntimeException(e); + } + } + + public static boolean willVacuumingCauseIssue(final JdbcDatabase database, final List streams, final String quoteString) { + final List streamsUnderVacuuming = new ArrayList<>(); + streams.forEach(stream -> { final String streamName = stream.getStream().getName(); final String schemaName = stream.getStream().getNamespace(); final String fullTableName = getFullyQualifiedTableNameWithQuoting(schemaName, streamName, quoteString); - LOGGER.info("Full Vacuum information for {}", fullTableName); try { - List jsonNodes = database.bufferedResultSetQuery( - conn -> conn.prepareStatement(CTID_FULL_VACUUM_REL_FILENODE_QUERY.formatted(fullTableName)).executeQuery(), + final List jsonNodes = database.bufferedResultSetQuery( + conn -> conn.prepareStatement(CTID_FULL_VACUUM_IN_PROGRESS_QUERY.formatted(fullTableName)).executeQuery(), resultSet -> JdbcUtils.getDefaultSourceOperations().rowToJson(resultSet)); - Preconditions.checkState(jsonNodes.size() == 1); - LOGGER.info("Relation filenode is {}", jsonNodes.get(0).get("pg_relation_filenode")); - - jsonNodes = - database.bufferedResultSetQuery(conn -> conn.prepareStatement(CTID_FULL_VACUUM_IN_PROGRESS_QUERY.formatted(fullTableName)).executeQuery(), - resultSet -> JdbcUtils.getDefaultSourceOperations().rowToJson(resultSet)); - if (jsonNodes.size() == 0) { - LOGGER.info("No full vacuum currently in progress"); - } else { + if (jsonNodes.size() != 0) { Preconditions.checkState(jsonNodes.size() == 1); - LOGGER.info("Full Vacuum currently in progress in {} phase", jsonNodes.get(0).get("phase")); + LOGGER.warn("Full Vacuum currently in progress for table {} in {} phase", fullTableName, jsonNodes.get(0).get("phase")); + streamsUnderVacuuming.add(fullTableName); } } catch (SQLException e) { - LOGGER.warn("Failed to log full vacuum in progress. This warning shouldn't affect the sync and can be ignored", e); + // Assume it's safe to progress and skip relation node and vaccuum validation + LOGGER.warn("Failed to fetch vacuum for table {} info. Going to move ahead with the sync assuming it's safe", fullTableName, e); } }); + return !streamsUnderVacuuming.isEmpty(); } } 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 1308413f39c6fb..6d8c8d26df45f0 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 @@ -20,6 +20,8 @@ import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.ROW_COUNT_RESULT_COL; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TABLE_ESTIMATE_QUERY; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TOTAL_BYTES_RESULT_COL; +import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.fileNode; +import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.willVacuumingCauseIssue; import static io.airbyte.integrations.source.postgres.PostgresUtils.isIncrementalSyncMode; import static io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.categoriseStreams; import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; @@ -33,7 +35,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; -import com.google.common.collect.Sets; import datadog.trace.api.Trace; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.commons.features.EnvVariableFeatureFlags; @@ -74,13 +75,11 @@ import io.airbyte.protocol.models.v0.AirbyteConnectionStatus.Status; import io.airbyte.protocol.models.v0.AirbyteEstimateTraceMessage.Type; import io.airbyte.protocol.models.v0.AirbyteMessage; -import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.v0.AirbyteStream; import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; -import io.airbyte.protocol.models.v0.StreamDescriptor; import java.net.URI; import java.net.URISyntaxException; import java.nio.file.Path; @@ -93,7 +92,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; @@ -228,7 +226,6 @@ protected Set getExcludedViews() { protected void logPreSyncDebugData(final JdbcDatabase database, final ConfiguredAirbyteCatalog catalog) throws SQLException { super.logPreSyncDebugData(database, catalog); - PostgresQueryUtils.logFullVacuumStatus(database, catalog, getQuoteString()); for (final ConfiguredAirbyteStream stream : catalog.getStreams()) { final String streamName = stream.getStream().getName(); final String schemaName = stream.getStream().getNamespace(); @@ -472,10 +469,17 @@ public List> getIncrementalIterators(final final List> xminIterator = new ArrayList<>(); if (!streamsCategorised.ctidStreams.streamsForCtidSync.isEmpty()) { - final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams.statesFromCtidSync); - final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(database, sourceOperations, getQuoteString(), ctidStateManager, - x -> new ObjectMapper().valueToTree(xminStatus), - (pair, jsonState) -> XminStateManager.getAirbyteStateMessage(pair, Jsons.object(jsonState, XminStatus.class))); + if (willVacuumingCauseIssue(database, streamsCategorised.ctidStreams.streamsForCtidSync, getQuoteString())) { + //TODO : should we just skip these streams instead of failing the entire sync and just LOG warning stating that we are skipping these streams + throw new RuntimeException("Stopping sync cause few streams are under Vacuuming, can not sync them using ctid"); + } + + final Map fileNodes = fileNode(database, + streamsCategorised.ctidStreams.streamsForCtidSync, getQuoteString()); + final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams.statesFromCtidSync, fileNodes); + final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, sourceOperations, getQuoteString(), fileNodes, ctidStateManager, + namespacePair -> new ObjectMapper().valueToTree(xminStatus), + (namespacePair, jsonState) -> XminStateManager.getAirbyteStateMessage(namespacePair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams.streamsForCtidSync), tableNameToTable, emittedAt)); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 2623ebf65c300e..3df7dcde6bdbd0 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -10,40 +10,54 @@ import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteMessage.Type; import io.airbyte.protocol.models.v0.AirbyteStateMessage; +import java.time.Duration; +import java.time.Instant; +import java.time.OffsetDateTime; import java.util.Iterator; -import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import javax.annotation.CheckForNull; import org.apache.commons.lang3.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -public class CtidStateIterator extends AbstractIterator implements Iterator -{ +public class CtidStateIterator extends AbstractIterator implements Iterator { private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateIterator.class); + public static final Duration SYNC_CHECKPOINT_DURATION = Duration.ofMinutes(15); + public static final Integer SYNC_CHECKPOINT_RECORDS = 10_000; + + private final Iterator messageIterator; private final AirbyteStreamNameNamespacePair pair; private boolean hasEmittedFinalState; private boolean hasCaughtException = false; private String lastCtid; private final JsonNode streamStateForIncrementalRun; - final BiFunction finalStateMessageSupplier; - final AtomicLong recordCount = new AtomicLong(); + private final long relationFileNode; + private final BiFunction finalStateMessageSupplier; + private long recordCount = 0L; + private Instant lastCheckpoint = Instant.now(); + private final Duration syncCheckpointDuration; + private final Long syncCheckpointRecords; public CtidStateIterator(final Iterator messageIterator, final AirbyteStreamNameNamespacePair pair, + final long relationFileNode, final JsonNode streamStateForIncrementalRun, - final BiFunction finalStateMessageSupplier) { + final BiFunction finalStateMessageSupplier, + final Duration checkpointDuration, + final Long checkpointRecords) { this.messageIterator = messageIterator; this.pair = pair; + this.relationFileNode = relationFileNode; this.streamStateForIncrementalRun = streamStateForIncrementalRun; this.finalStateMessageSupplier = finalStateMessageSupplier; + this.syncCheckpointDuration = checkpointDuration; + this.syncCheckpointRecords = checkpointRecords; } @CheckForNull @Override protected AirbyteMessage computeNext() { - final long count = recordCount.incrementAndGet(); if (hasCaughtException) { // Mark iterator as done since the next call to messageIterator will result in an // IllegalArgumentException and resets exception caught state. @@ -54,15 +68,18 @@ protected AirbyteMessage computeNext() { } if (messageIterator.hasNext()) { - if (count % 1_000_000 == 0 && StringUtils.isNotBlank(lastCtid)) { - LOGGER.info("saving ctid state with {}", this.lastCtid); - //TODO (Rodi): To add relation_filenode attribute in the CtidStatus - return CtidStateManager.createPerStreamStateMessage(pair, - new CtidStatus() - .withVersion(CTID_STATUS_VERSION) - .withStateType(StateType.CTID) - .withCtid(lastCtid) - .withIncrementalState(streamStateForIncrementalRun)); + if ((recordCount >= syncCheckpointRecords || Duration.between(lastCheckpoint, OffsetDateTime.now()).compareTo(syncCheckpointDuration) > 0) + && StringUtils.isNotBlank(lastCtid)) { + final CtidStatus ctidStatus = new CtidStatus() + .withVersion(CTID_STATUS_VERSION) + .withStateType(StateType.CTID) + .withCtid(lastCtid) + .withIncrementalState(streamStateForIncrementalRun) + .withRelationFilenode(relationFileNode); + LOGGER.info("Emitting ctid state for stream {}, state is {}", pair, ctidStatus); + recordCount = 0L; + lastCheckpoint = Instant.now(); + return CtidStateManager.createPerStreamStateMessage(pair, ctidStatus); } // Use try-catch to catch Exception that could occur when connection to the database fails try { @@ -70,14 +87,10 @@ protected AirbyteMessage computeNext() { if (message.getRecord().getData().hasNonNull("ctid")) { this.lastCtid = message.getRecord().getData().get("ctid").asText(); } + recordCount++; return message; } catch (final Exception e) { - hasCaughtException = true; - LOGGER.error("Message iterator failed to read next record.", e); - // We want to still continue attempting to sync future streams, so the exception is caught. When - // frequent state emission is introduced, this - // will result in a partial success. - return endOfData(); + throw new RuntimeException(e); } } else if (!hasEmittedFinalState) { hasEmittedFinalState = true; diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 0cfc642f0bf8e8..78afdf01b7be11 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -15,6 +15,7 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -26,11 +27,12 @@ public class CtidStateManager { .withType(AirbyteStateType.STREAM) .withStream(new AirbyteStreamState()); - public CtidStateManager(final List stateMessages) { - this.pairToCtidStatus = createPairToCtidStatusMap(stateMessages); + public CtidStateManager(final List stateMessages, final Map fileNodes) { + this.pairToCtidStatus = createPairToCtidStatusMap(stateMessages, fileNodes); } - private static Map createPairToCtidStatusMap(final List stateMessages) { + private static Map createPairToCtidStatusMap(final List stateMessages, + final Map fileNodes) { final Map localMap = new HashMap<>(); if (stateMessages != null) { for (final AirbyteStateMessage stateMessage : stateMessages) { @@ -42,17 +44,27 @@ private static Map createPairToCtidS try { ctidStatus = Jsons.object(stateMessage.getStream().getStreamState(), CtidStatus.class); assert (ctidStatus.getVersion() == CTID_STATUS_VERSION); - assert(ctidStatus.getStateType().equals(StateType.CTID)); // TODO: check here + assert (ctidStatus.getStateType().equals(StateType.CTID)); } catch (final IllegalArgumentException e) { throw new ConfigErrorException("Invalid per-stream state"); } - localMap.put(pair, ctidStatus); + if (validateRelationFileNode(ctidStatus, pair, fileNodes)) { + localMap.put(pair, ctidStatus); + } } } } return localMap; } + private static boolean validateRelationFileNode(final CtidStatus ctidstatus, final AirbyteStreamNameNamespacePair pair, final Map fileNodes) { + if (fileNodes.containsKey(pair)) { + final Long fileNode = fileNodes.get(pair); + return Objects.equals(ctidstatus.getRelationFilenode(), fileNode); + } + return true; + } + public CtidStatus getCtidStatus(final AirbyteStreamNameNamespacePair pair) { return pairToCtidStatus.get(pair); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index e371e638285129..2b960c2ac89c77 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -28,6 +28,7 @@ import java.sql.Connection; import java.sql.PreparedStatement; import java.sql.SQLException; +import java.time.Duration; import java.time.Instant; import java.util.ArrayList; import java.util.List; @@ -35,7 +36,6 @@ import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Stream; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -43,22 +43,29 @@ public class PostgresCtidHandler { private static final Logger LOGGER = LoggerFactory.getLogger(PostgresCtidHandler.class); + + private final JsonNode config; private final JdbcDatabase database; private final JdbcCompatibleSourceOperations sourceOperations; private final String quoteString; private final CtidStateManager ctidStateManager; + private final Map fileNodes; private final Function streamStateForIncrementalRunSupplier; private final BiFunction finalStateMessageSupplier; - public PostgresCtidHandler(final JdbcDatabase database, + public PostgresCtidHandler(final JsonNode config, + final JdbcDatabase database, final JdbcCompatibleSourceOperations sourceOperations, final String quoteString, + final Map fileNodes, final CtidStateManager ctidStateManager, final Function streamStateForIncrementalRunSupplier, final BiFunction finalStateMessageSupplier) { + this.config = config; this.database = database; this.sourceOperations = sourceOperations; this.quoteString = quoteString; + this.fileNodes = fileNodes; this.ctidStateManager = ctidStateManager; this.streamStateForIncrementalRunSupplier = streamStateForIncrementalRunSupplier; this.finalStateMessageSupplier = finalStateMessageSupplier; @@ -187,10 +194,18 @@ private AutoCloseableIterator augmentWithState(final AutoCloseab final JsonNode incrementalState = (currentCtidStatus == null || currentCtidStatus.getIncrementalState() == null) ? streamStateForIncrementalRunSupplier.apply(pair) : currentCtidStatus.getIncrementalState(); + final Long latestFileNode = fileNodes.get(pair); + assert latestFileNode != null; + final Duration syncCheckpointDuration = + config.get("sync_checkpoint_seconds") != null ? Duration.ofSeconds(config.get("sync_checkpoint_seconds").asLong()) + : CtidStateIterator.SYNC_CHECKPOINT_DURATION; + final Long syncCheckpointRecords = config.get("sync_checkpoint_records") != null ? config.get("sync_checkpoint_records").asLong() + : CtidStateIterator.SYNC_CHECKPOINT_RECORDS; return AutoCloseableIterators.transform( - autoClosableIterator -> new CtidStateIterator(recordIterator, pair, incrementalState, finalStateMessageSupplier ), recordIterator, pair); + autoClosableIterator -> new CtidStateIterator(recordIterator, pair, latestFileNode, incrementalState, finalStateMessageSupplier, + syncCheckpointDuration, syncCheckpointRecords), recordIterator, pair); } private AutoCloseableIterator swallowCtid(final AutoCloseableIterator iterator, diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index 6e78378c259dbb..4de29c06cbd823 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -33,7 +33,7 @@ public static StreamsCategorised categoriseStreams(final StateManager stateManag return; } final AirbyteStateMessage clonedState = Jsons.clone(s); - if (streamState.has("type") && streamState.get("type").asText().equalsIgnoreCase("ctid")) { + if (streamState.has("state_type") && streamState.get("state_type").asText().equalsIgnoreCase("ctid")) { statesFromCtidSync.add(clonedState); streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); } else { From 7ef08e3b0892b8b45586714444af540f73343b1c Mon Sep 17 00:00:00 2001 From: subodh Date: Sun, 18 Jun 2023 16:41:19 +0530 Subject: [PATCH 11/27] refine test + make PR ready for review --- .../source/postgres/PostgresSource.java | 3 +- .../postgres/ctid/CtidStateManager.java | 3 +- .../postgres/xmin/XminStateManager.java | 3 +- .../postgres/XminPostgresSourceTest.java | 137 ++++++++++++++---- 4 files changed, 109 insertions(+), 37 deletions(-) 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 6d8c8d26df45f0..9afc5eb0c5e199 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 @@ -30,7 +30,6 @@ import static java.util.stream.Collectors.toSet; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; @@ -478,7 +477,7 @@ public List> getIncrementalIterators(final streamsCategorised.ctidStreams.streamsForCtidSync, getQuoteString()); final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams.statesFromCtidSync, fileNodes); final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, sourceOperations, getQuoteString(), fileNodes, ctidStateManager, - namespacePair -> new ObjectMapper().valueToTree(xminStatus), + namespacePair -> Jsons.jsonNode(xminStatus), (namespacePair, jsonState) -> XminStateManager.getAirbyteStateMessage(namespacePair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams.streamsForCtidSync), tableNameToTable, emittedAt)); diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 78afdf01b7be11..63cdb04cb31276 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -1,6 +1,5 @@ package io.airbyte.integrations.source.postgres.ctid; -import com.fasterxml.jackson.databind.ObjectMapper; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; @@ -77,7 +76,7 @@ public static AirbyteMessage createPerStreamStateMessage(final AirbyteStreamName new StreamDescriptor() .withName(pair.getName()) .withNamespace(pair.getNamespace())) - .withStreamState(new ObjectMapper().valueToTree(ctidStatus)); + .withStreamState(Jsons.jsonNode(ctidStatus)); final AirbyteStateMessage stateMessage = new AirbyteStateMessage() diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java index d2cd16a39ef367..a883a11d5469cc 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java @@ -4,7 +4,6 @@ package io.airbyte.integrations.source.postgres.xmin; -import com.fasterxml.jackson.databind.ObjectMapper; import io.airbyte.commons.exceptions.ConfigErrorException; import io.airbyte.commons.json.Jsons; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; @@ -87,7 +86,7 @@ public static AirbyteStateMessage getAirbyteStateMessage(final AirbyteStreamName new StreamDescriptor() .withName(pair.getName()) .withNamespace(pair.getNamespace())) - .withStreamState(new ObjectMapper().valueToTree(xminStatus)); + .withStreamState(Jsons.jsonNode(xminStatus)); // Set state final AirbyteStateMessage stateMessage = diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java index ee1ec29ae5fd7e..e8d9fe169fe0fc 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java @@ -9,12 +9,12 @@ import static io.airbyte.integrations.source.postgres.utils.PostgresUnitTestsUtil.setEmittedAtToNull; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; +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 com.google.common.collect.Lists; -import com.google.common.collect.Sets; import io.airbyte.commons.features.EnvVariableFeatureFlags; import io.airbyte.commons.io.IOs; import io.airbyte.commons.json.Jsons; @@ -38,9 +38,12 @@ import io.airbyte.protocol.models.v0.SyncMode; import io.airbyte.test.utils.PostgreSQLContainerHelper; import java.math.BigDecimal; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; import java.util.List; import java.util.Optional; -import java.util.Set; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import org.jooq.DSLContext; import org.jooq.SQLDialect; @@ -92,12 +95,12 @@ class XminPostgresSourceTest { private static final ConfiguredAirbyteCatalog CONFIGURED_XMIN_CATALOG = toConfiguredXminCatalog(CATALOG); - private static final Set INITIAL_RECORD_MESSAGES = Sets.newHashSet( + private static final List INITIAL_RECORD_MESSAGES = Arrays.asList( createRecord(STREAM_NAME, SCHEMA_NAME, map("id", new BigDecimal("1.0"), "name", "goku", "power", null)), createRecord(STREAM_NAME, SCHEMA_NAME, map("id", new BigDecimal("2.0"), "name", "vegeta", "power", 9000.1)), createRecord(STREAM_NAME, SCHEMA_NAME, map("id", null, "name", "piccolo", "power", null))); - private static final Set NEXT_RECORD_MESSAGES = Sets.newHashSet( + private static final List NEXT_RECORD_MESSAGES = Arrays.asList( createRecord(STREAM_NAME, SCHEMA_NAME, map("id", new BigDecimal("3.0"), "name", "gohan", "power", 222.1))); private static PostgreSQLContainer PSQL_DB; @@ -128,7 +131,7 @@ void setup() throws Exception { "CREATE TABLE id_and_name(id NUMERIC(20, 10) NOT NULL, name VARCHAR(200) NOT NULL, power double precision NOT NULL, PRIMARY KEY (id));"); ctx.fetch("CREATE INDEX i1 ON id_and_name (id);"); ctx.fetch( - "INSERT INTO id_and_name (id, name, power) VALUES (2, 'vegeta', 9000.1), (1,'goku', 'Infinity'), ('NaN', 'piccolo', '-Infinity');"); + "INSERT INTO id_and_name (id, name, power) VALUES (1,'goku', 'Infinity'), (2, 'vegeta', 9000.1), ('NaN', 'piccolo', '-Infinity');"); ctx.fetch("CREATE TABLE id_and_name2(id NUMERIC(20, 10) NOT NULL, name VARCHAR(200) NOT NULL, power double precision NOT NULL);"); ctx.fetch( @@ -169,6 +172,7 @@ private JsonNode getXminConfig(final PostgreSQLContainer psqlDb, final String .put(JdbcUtils.PASSWORD_KEY, psqlDb.getPassword()) .put(JdbcUtils.SSL_KEY, false) .put("replication_method", getReplicationMethod()) + .put("sync_checkpoint_records", 1) .build()); } @@ -202,30 +206,87 @@ void testReadSuccess() throws Exception { CONFIGURED_XMIN_CATALOG .withStreams(CONFIGURED_XMIN_CATALOG.getStreams().stream().filter(s -> s.getStream().getName().equals(STREAM_NAME)).collect( Collectors.toList())); - final List actualMessages = + final List recordsFromFirstSync = MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, null)); - setEmittedAtToNull(actualMessages); - assertThat(filterRecords(actualMessages)).containsExactlyInAnyOrderElementsOf(INITIAL_RECORD_MESSAGES); + setEmittedAtToNull(recordsFromFirstSync); + assertThat(filterRecords(recordsFromFirstSync)).containsExactlyElementsOf(INITIAL_RECORD_MESSAGES); // Extract the state message and assert that it exists. It contains the xmin value, so validating // the actual value isn't useful right now. - final List stateAfterFirstBatch = extractStateMessage(actualMessages); - assertThat(stateAfterFirstBatch.size()).isEqualTo(1); - JsonNode state = Jsons.jsonNode(stateAfterFirstBatch); + final List stateAfterFirstBatch = extractStateMessage(recordsFromFirstSync); + // We should have 3 state messages because we have set state emission frequency after each record in the test + assertEquals(3, stateAfterFirstBatch.size()); + + final AirbyteStateMessage firstStateMessage = stateAfterFirstBatch.get(0); + final String stateTypeFromFirstStateMessage = firstStateMessage.getStream().getStreamState().get("state_type").asText(); + final String ctidFromFirstStateMessage = firstStateMessage.getStream().getStreamState().get("ctid").asText(); + final JsonNode incrementalStateFromFirstStateMessage = firstStateMessage.getStream().getStreamState().get("incremental_state"); + + final AirbyteStateMessage secondStateMessage = stateAfterFirstBatch.get(1); + final String stateTypeFromSecondStateMessage = secondStateMessage.getStream().getStreamState().get("state_type").asText(); + final String ctidFromSecondStateMessage = secondStateMessage.getStream().getStreamState().get("ctid").asText(); + final JsonNode incrementalStateFromSecondStateMessage = secondStateMessage.getStream().getStreamState().get("incremental_state"); + + final AirbyteStateMessage thirdStateMessage = stateAfterFirstBatch.get(2); + final String stateTypeFromThirdStateMessage = thirdStateMessage.getStream().getStreamState().get("state_type").asText(); + + // First two state messages should be of ctid type + assertEquals("ctid", stateTypeFromFirstStateMessage); + assertEquals("ctid", stateTypeFromSecondStateMessage); + + // Since the third state message would be the final, it should be of xmin type + assertEquals("xmin", stateTypeFromThirdStateMessage); + + // The ctid value from second state message should be bigger than first state message + assertEquals(1, ctidFromSecondStateMessage.compareTo(ctidFromFirstStateMessage)); + + // The incremental state value from first and second state message should be the same + assertNotNull(incrementalStateFromFirstStateMessage); + assertNotNull(incrementalStateFromSecondStateMessage); + assertEquals(incrementalStateFromFirstStateMessage, incrementalStateFromSecondStateMessage); + + // The third state message should be equal to incremental_state of first two state messages + assertEquals(incrementalStateFromFirstStateMessage, thirdStateMessage.getStream().getStreamState()); // Assert that the last message in the sequence is a state message - assertMessageSequence(actualMessages); + assertMessageSequence(recordsFromFirstSync); + + // Sync should work with a ctid state + final List recordsFromSyncRunningWithACtidState = + MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, + Jsons.jsonNode(Collections.singletonList(firstStateMessage)))); + setEmittedAtToNull(recordsFromSyncRunningWithACtidState); + final List expectedDataFromSyncUsingFirstCtidState = new ArrayList<>(2); + final AtomicBoolean skippedFirstRecord = new AtomicBoolean(false); + INITIAL_RECORD_MESSAGES.forEach(c -> { + if (!skippedFirstRecord.get()) { + skippedFirstRecord.set(true); + return; + } + expectedDataFromSyncUsingFirstCtidState.add(c); + }); + assertThat(filterRecords(recordsFromSyncRunningWithACtidState)).containsExactlyElementsOf(expectedDataFromSyncUsingFirstCtidState); + + final List stateAfterSyncWithCtidState = extractStateMessage(recordsFromSyncRunningWithACtidState); + // Since only 2 records should be emitted so 2 state messages are expected + assertEquals(2, stateAfterSyncWithCtidState.size()); + assertEqualsCtidState(secondStateMessage.getStream().getStreamState(), stateAfterSyncWithCtidState.get(0).getStream().getStreamState()); + assertEquals(thirdStateMessage, stateAfterSyncWithCtidState.get(1)); + + assertMessageSequence(recordsFromSyncRunningWithACtidState); - // Second read, should return no data - final List nextMessages = - MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, state)); - setEmittedAtToNull(nextMessages); - assertThat(filterRecords(nextMessages)).isEmpty(); + // Read with the final xmin state message should return no data + final List syncWithXminStateType = + MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, + Jsons.jsonNode(Collections.singletonList(thirdStateMessage)))); + setEmittedAtToNull(syncWithXminStateType); + assertEquals(0, filterRecords(syncWithXminStateType).size()); // Even though no records were emitted, a state message is still expected - final List stateAfterSecondBatch = extractStateMessage(nextMessages); - assertThat(stateAfterFirstBatch.size()).isEqualTo(1); - state = Jsons.jsonNode(stateAfterSecondBatch); + final List stateAfterXminSync = extractStateMessage(syncWithXminStateType); + assertEquals(1, stateAfterXminSync.size()); + // Since no records were returned so the state should be the same as before + assertEquals(thirdStateMessage, stateAfterXminSync.get(0)); // We add some data and perform a third read. We should verify that (i) a delete is not captured and // (ii) the new record that is inserted into the @@ -239,21 +300,35 @@ void testReadSuccess() throws Exception { }); } - final List lastMessages = - MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, state)); - setEmittedAtToNull(lastMessages); - assertThat(filterRecords(lastMessages)).containsExactlyInAnyOrderElementsOf(NEXT_RECORD_MESSAGES); - assertMessageSequence(lastMessages); + final List recordsAfterLastSync = + MoreIterators.toList(new PostgresSource().read(getXminConfig(PSQL_DB, dbName), configuredCatalog, + Jsons.jsonNode(Collections.singletonList(stateAfterXminSync.get(0))))); + setEmittedAtToNull(recordsAfterLastSync); + assertThat(filterRecords(recordsAfterLastSync)).containsExactlyElementsOf(NEXT_RECORD_MESSAGES); + assertMessageSequence(recordsAfterLastSync); + final List stateAfterLastSync = extractStateMessage(recordsAfterLastSync); + assertEquals(1, stateAfterLastSync.size()); + } + + private void assertEqualsCtidState(final JsonNode left, final JsonNode right) { + assertEquals(left.get("version").asText(), right.get("version").asText()); + assertEquals(left.get("state_type").asText(), right.get("state_type").asText()); + assertEquals(left.get("ctid").asText(), right.get("ctid").asText()); + assertEqualsXminState(left.get("incremental_state"), right.get("incremental_state")); + assertEquals(left.get("relation_filenode"), right.get("relation_filenode")); + } + + private void assertEqualsXminState(final JsonNode left, final JsonNode right) { + assertEquals(left.get("version").asText(), right.get("version").asText()); + assertEquals(left.get("state_type").asText(), right.get("state_type").asText()); + assertEquals(left.get("num_wraparound").asText(), right.get("num_wraparound").asText()); + assertEquals(left.get("xmin_xid_value").asText(), right.get("xmin_xid_value").asText()); + assertEquals(left.get("xmin_raw_value").asText(), right.get("xmin_raw_value").asText()); } // Assert that the state message is the last message to be emitted. private static void assertMessageSequence(final List messages) { - for (int i = 0; i < messages.size(); i++) { - final AirbyteMessage message = messages.get(i); - if (message.getType().equals(Type.STATE)) { - assertThat(i).isEqualTo(messages.size() - 1); - } - } + assertEquals(Type.STATE, messages.get(messages.size() - 1).getType()); } private static List extractStateMessage(final List messages) { From cba94db1603e246f53191606c0d1e7a2028d1e9a Mon Sep 17 00:00:00 2001 From: subodh Date: Sun, 18 Jun 2023 16:46:25 +0530 Subject: [PATCH 12/27] remove un-wanted changes --- .../io/airbyte/db/factory/DataSourceFactory.java | 1 - .../io/airbyte/db/jdbc/StreamingJdbcDatabase.java | 13 ------------- .../source/relationaldb/AbstractDbSource.java | 4 ++-- deps.toml | 2 +- 4 files changed, 3 insertions(+), 17 deletions(-) diff --git a/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java b/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java index 384e8cfa37b31a..d970b09fbba959 100644 --- a/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java +++ b/airbyte-db/db-lib/src/main/java/io/airbyte/db/factory/DataSourceFactory.java @@ -306,7 +306,6 @@ public DataSource build() { config.setPassword(password); config.setUsername(username); - config.setIdleTimeout(1_000); // TODO: check here /* * Disable to prevent failing on startup. Applications may start prior to the database container * being available. To avoid failing to create the connection pool, disable the fail check. This diff --git a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java index 481906f41734fc..502cce75d737b8 100644 --- a/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java +++ b/airbyte-db/db-lib/src/main/java/io/airbyte/db/jdbc/StreamingJdbcDatabase.java @@ -95,19 +95,13 @@ protected Stream toUnsafeStream(final ResultSet resultSet, @Override public boolean tryAdvance(final Consumer action) { try { - // LOGGER.info("tryAdvance"); - // sleep(3000); if (!resultSet.next()) { - // LOGGER.error("exiting tryAdvance"); - // LOGGER.error("stacktrace: \n{}", Arrays.toString(Thread.currentThread().getStackTrace())); resultSet.close(); - // LOGGER.info("return f1"); return false; } final T dataRow = mapper.apply(resultSet); streamingConfig.accept(resultSet, dataRow); action.accept(dataRow); - // LOGGER.info("return t2 {}", dataRow.toString()); return true; } catch (final SQLException e) { LOGGER.error("SQLState: {}, Message: {}", e.getSQLState(), e.getMessage()); @@ -115,13 +109,6 @@ public boolean tryAdvance(final Consumer action) { isStreamFailed = true; // throwing an exception in tryAdvance() method lead to the endless loop in Spliterator and stream // will never close - // LOGGER.info("return f3"); - return false; - } catch (final Exception e) { - LOGGER.error("General exception", e); - streamException = e; - isStreamFailed = true; - // LOGGER.info("return f4"); return false; } } diff --git a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java index 591058ea48280b..b805587ba2d9e1 100644 --- a/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java +++ b/airbyte-integrations/connectors/source-relational-db/src/main/java/io/airbyte/integrations/source/relationaldb/AbstractDbSource.java @@ -418,8 +418,8 @@ private AutoCloseableIterator createReadIterator(final Database AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()), r -> { final long count = recordCount.incrementAndGet(); - if (count % 1_000_000 == 0) { - LOGGER.info("Reading stream {}. /airbyte-config/init/: {}", streamName, count); + if (count % 10000 == 0) { + LOGGER.info("Reading stream {}. Records read: {}", streamName, count); } return r; }); diff --git a/deps.toml b/deps.toml index c4077a3ded95d3..42ee78ea9ce40e 100644 --- a/deps.toml +++ b/deps.toml @@ -27,7 +27,7 @@ micronaut-jaxrs = "3.4.0" micronaut-security = "3.9.2" micronaut-test = "3.8.0" platform-testcontainers = "1.17.3" -postgresql = "42.6.0" +postgresql = "42.3.5" reactor = "3.5.2" segment = "2.1.1" slf4j = "1.7.36" From 378357ab18048d464815742367dd87c9f821a865 Mon Sep 17 00:00:00 2001 From: subodh Date: Sun, 18 Jun 2023 17:00:06 +0530 Subject: [PATCH 13/27] missed this one --- .../source/postgres/ctid/CtidStateIterator.java | 10 ---------- 1 file changed, 10 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 3df7dcde6bdbd0..d781d6bcd2f70f 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -29,7 +29,6 @@ public class CtidStateIterator extends AbstractIterator implemen private final Iterator messageIterator; private final AirbyteStreamNameNamespacePair pair; private boolean hasEmittedFinalState; - private boolean hasCaughtException = false; private String lastCtid; private final JsonNode streamStateForIncrementalRun; private final long relationFileNode; @@ -58,15 +57,6 @@ public CtidStateIterator(final Iterator messageIterator, @CheckForNull @Override protected AirbyteMessage computeNext() { - if (hasCaughtException) { - // Mark iterator as done since the next call to messageIterator will result in an - // IllegalArgumentException and resets exception caught state. - // This occurs when the previous iteration emitted state so this iteration cycle will indicate - // iteration is complete - hasCaughtException = false; - return endOfData(); - } - if (messageIterator.hasNext()) { if ((recordCount >= syncCheckpointRecords || Duration.between(lastCheckpoint, OffsetDateTime.now()).compareTo(syncCheckpointDuration) > 0) && StringUtils.isNotBlank(lastCtid)) { From cb5bd39a15cec45eb24313dcb0bec280d2e5c8bb Mon Sep 17 00:00:00 2001 From: subodh Date: Sun, 18 Jun 2023 20:51:39 +0530 Subject: [PATCH 14/27] remove irrelevant comments --- .../source/postgres/ctid/PostgresCtidHandler.java | 4 ---- 1 file changed, 4 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index 2b960c2ac89c77..8b21d2828e7c8a 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -81,7 +81,6 @@ public List> getIncrementalIterators( final String namespace = stream.getNamespace(); final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamName, namespace); final String fullyQualifiedTableName = DbSourceDiscoverUtil.getFullyQualifiedTableName(namespace, streamName); - //TODO: what if stream doesn't need ctid if (!tableNameToTable.containsKey(fullyQualifiedTableName)) { LOGGER.info("Skipping stream {} because it is not in the source", fullyQualifiedTableName); continue; @@ -137,9 +136,6 @@ private PreparedStatement createCtidQueryStatement( quoteString); final String wrappedColumnNames = RelationalDbQueryUtils.enquoteIdentifierList(columnNames, quoteString); - // The xmin state that we save represents the lowest XID that is still in progress. To make sure we - // don't miss - // data associated with the current transaction, we have to issue an >= final String sql = "SELECT ctid, %s FROM %s WHERE ctid > ?::tid".formatted(wrappedColumnNames, fullTableName); final PreparedStatement preparedStatement = connection.prepareStatement(sql); final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(airbyteStream); From 0689f3089687771bc772322ec5f434625ef067de Mon Sep 17 00:00:00 2001 From: subodh Date: Mon, 19 Jun 2023 14:08:07 +0530 Subject: [PATCH 15/27] add more assertions --- .../source/postgres/XminPostgresSourceTest.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java index e8d9fe169fe0fc..1760acb65cb9f3 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java @@ -308,6 +308,14 @@ void testReadSuccess() throws Exception { assertMessageSequence(recordsAfterLastSync); final List stateAfterLastSync = extractStateMessage(recordsAfterLastSync); assertEquals(1, stateAfterLastSync.size()); + + final AirbyteStateMessage finalStateMesssage = stateAfterLastSync.get(0); + final String stateTypeFromFinalStateMessage = finalStateMesssage.getStream().getStreamState().get("state_type").asText(); + assertEquals("xmin", stateTypeFromFinalStateMessage); + assertTrue(finalStateMesssage.getStream().getStreamState().get("xmin_xid_value").asLong() > thirdStateMessage.getStream().getStreamState() + .get("xmin_xid_value").asLong()); + assertTrue(finalStateMesssage.getStream().getStreamState().get("xmin_raw_value").asLong() > thirdStateMessage.getStream().getStreamState() + .get("xmin_raw_value").asLong()); } private void assertEqualsCtidState(final JsonNode left, final JsonNode right) { From 7ea3fdf61b595f1cadbbd35c4493a2a3b75974ae Mon Sep 17 00:00:00 2001 From: subodh Date: Mon, 19 Jun 2023 15:05:48 +0530 Subject: [PATCH 16/27] remove jdbc log --- .../airbyte/integrations/source/postgres/PostgresSource.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) 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 9afc5eb0c5e199..602de0462a56ed 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 @@ -175,8 +175,7 @@ public JsonNode toDatabaseConfig(final JsonNode config) { additionalParameters.forEach(x -> jdbcUrl.append(x).append("&")); jdbcUrl.append(toJDBCQueryParams(sslParameters)); -// jdbcUrl.append("&options=-c%20statement_timeout=15000"); - LOGGER.info("jdbc url: {}", jdbcUrl.toString()); + LOGGER.debug("jdbc url: {}", jdbcUrl.toString()); final ImmutableMap.Builder configBuilder = ImmutableMap.builder() .put(JdbcUtils.USERNAME_KEY, config.get(JdbcUtils.USERNAME_KEY).asText()) .put(JdbcUtils.JDBC_URL_KEY, jdbcUrl.toString()); From f7e9939279d2568bbe180eee5b10de4d66565d46 Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 20 Jun 2023 14:31:53 +0530 Subject: [PATCH 17/27] address review comments --- .../commons/util/AutoCloseableIterators.java | 6 +++ .../source/postgres/PostgresQueryUtils.java | 6 +-- .../source/postgres/PostgresSource.java | 28 ++++++----- .../postgres/ctid/AirbyteMessageWithCtid.java | 7 +++ .../ctid/CtidPostgresSourceOperations.java | 39 +++++++++++++++ .../postgres/ctid/CtidStateIterator.java | 14 +++--- .../postgres/ctid/PostgresCtidHandler.java | 47 +++++++------------ .../source/postgres/xmin/XminCtidUtils.java | 46 +++++++----------- 8 files changed, 111 insertions(+), 82 deletions(-) create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java create mode 100644 airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java 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 26152d96247973..5a66d803eae835 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 @@ -174,6 +174,12 @@ public static AutoCloseableIterator transform(final Function(iteratorCreator.apply(autoCloseableIterator), autoCloseableIterator::close, airbyteStream); } + public static AutoCloseableIterator transformIterator(final Function, Iterator> iteratorCreator, + final AutoCloseableIterator autoCloseableIterator, + final AirbyteStreamNameNamespacePair airbyteStream) { + return new DefaultAutoCloseableIterator(iteratorCreator.apply(autoCloseableIterator), autoCloseableIterator::close, airbyteStream); + } + @SafeVarargs public static CompositeIterator concatWithEagerClose(final Consumer airbyteStreamStatusConsumer, final AutoCloseableIterator... iterators) { diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 83e36eee835588..7dbfc9400dcdfe 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -99,17 +99,17 @@ public static XminStatus getXminStatus(final JdbcDatabase database) throws SQLEx .withStateType(StateType.XMIN); } - public static Map fileNode(final JdbcDatabase database, final List streams, final String quoteString) { + static Map fileNodeForStreams(final JdbcDatabase database, final List streams, final String quoteString) { final Map fileNodes = new HashMap<>(); streams.forEach(stream -> { final AirbyteStreamNameNamespacePair namespacePair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()); - final long l = fileNode(database, namespacePair, quoteString); + final long l = fileNodeForStreams(database, namespacePair, quoteString); fileNodes.put(namespacePair, l); }); return fileNodes; } - public static long fileNode(final JdbcDatabase database, final AirbyteStreamNameNamespacePair stream, final String quoteString) { + public static long fileNodeForStreams(final JdbcDatabase database, final AirbyteStreamNameNamespacePair stream, final String quoteString) { try { final String streamName = stream.getName(); final String schemaName = stream.getNamespace(); 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 602de0462a56ed..e9424712ee659c 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 @@ -20,7 +20,7 @@ import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.ROW_COUNT_RESULT_COL; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TABLE_ESTIMATE_QUERY; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TOTAL_BYTES_RESULT_COL; -import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.fileNode; +import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.fileNodeForStreams; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.willVacuumingCauseIssue; import static io.airbyte.integrations.source.postgres.PostgresUtils.isIncrementalSyncMode; import static io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.categoriseStreams; @@ -59,6 +59,7 @@ import io.airbyte.integrations.source.jdbc.JdbcSSLConnectionUtils; import io.airbyte.integrations.source.jdbc.JdbcSSLConnectionUtils.SslMode; import io.airbyte.integrations.source.jdbc.dto.JdbcPrivilegeDto; +import io.airbyte.integrations.source.postgres.ctid.CtidPostgresSourceOperations; import io.airbyte.integrations.source.postgres.ctid.CtidStateManager; import io.airbyte.integrations.source.postgres.ctid.PostgresCtidHandler; import io.airbyte.integrations.source.postgres.internal.models.XminStatus; @@ -460,34 +461,35 @@ public List> getIncrementalIterators(final } else if (PostgresUtils.isXmin(sourceConfig) && isIncrementalSyncMode(catalog)) { final StreamsCategorised streamsCategorised = categoriseStreams(stateManager, catalog); - LOGGER.info("Streams to be synced via ctid : {}", streamsCategorised.ctidStreams.streamsForCtidSync.size()); - LOGGER.info("Streams to be synced via xmin : {}", streamsCategorised.xminStreams.streamsForXminSync.size()); + LOGGER.info("Streams to be synced via ctid : {}", streamsCategorised.ctidStreams().streamsForCtidSync().size()); + LOGGER.info("Streams to be synced via xmin : {}", streamsCategorised.xminStreams().streamsForXminSync().size()); final List> ctidIterator = new ArrayList<>(); final List> xminIterator = new ArrayList<>(); - if (!streamsCategorised.ctidStreams.streamsForCtidSync.isEmpty()) { - if (willVacuumingCauseIssue(database, streamsCategorised.ctidStreams.streamsForCtidSync, getQuoteString())) { + if (!streamsCategorised.ctidStreams().streamsForCtidSync().isEmpty()) { + if (willVacuumingCauseIssue(database, streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString())) { //TODO : should we just skip these streams instead of failing the entire sync and just LOG warning stating that we are skipping these streams throw new RuntimeException("Stopping sync cause few streams are under Vacuuming, can not sync them using ctid"); } - final Map fileNodes = fileNode(database, - streamsCategorised.ctidStreams.streamsForCtidSync, getQuoteString()); - final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams.statesFromCtidSync, fileNodes); - final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, sourceOperations, getQuoteString(), fileNodes, ctidStateManager, + final Map fileNodes = PostgresQueryUtils.fileNodeForStreams(database, + streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString()); + final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams().statesFromCtidSync(), fileNodes); + final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, new CtidPostgresSourceOperations(), getQuoteString(), + fileNodes, ctidStateManager, namespacePair -> Jsons.jsonNode(xminStatus), (namespacePair, jsonState) -> XminStateManager.getAirbyteStateMessage(namespacePair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams.streamsForCtidSync), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams().streamsForCtidSync()), tableNameToTable, emittedAt)); } - if (!streamsCategorised.xminStreams.streamsForXminSync.isEmpty()) { - final XminStateManager xminStateManager = new XminStateManager(streamsCategorised.xminStreams.statesFromXminSync); + if (!streamsCategorised.xminStreams().streamsForXminSync().isEmpty()) { + final XminStateManager xminStateManager = new XminStateManager(streamsCategorised.xminStreams().statesFromXminSync()); final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); xminIterator.addAll(xminHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.xminStreams.streamsForXminSync), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.xminStreams().streamsForXminSync()), tableNameToTable, emittedAt)); } return Stream diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java new file mode 100644 index 00000000000000..c961751580bdec --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java @@ -0,0 +1,7 @@ +package io.airbyte.integrations.source.postgres.ctid; + +import io.airbyte.protocol.models.v0.AirbyteMessage; + +public record AirbyteMessageWithCtid(AirbyteMessage recordMessage, String ctid) { + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java new file mode 100644 index 00000000000000..64aa470274c4b3 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java @@ -0,0 +1,39 @@ +package io.airbyte.integrations.source.postgres.ctid; + +import com.fasterxml.jackson.databind.JsonNode; +import com.fasterxml.jackson.databind.node.ObjectNode; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.postgres.PostgresSourceOperations; +import java.sql.ResultSet; +import java.sql.ResultSetMetaData; +import java.sql.SQLException; +import java.util.Collections; +import java.util.Objects; + +public class CtidPostgresSourceOperations extends PostgresSourceOperations { + + public RowDataWithCtid recordWithCtid(final ResultSet queryContext) throws SQLException { + // the first call communicates with the database. after that the result is cached. + final ResultSetMetaData metadata = queryContext.getMetaData(); + final int columnCount = metadata.getColumnCount(); + final ObjectNode jsonNode = (ObjectNode) Jsons.jsonNode(Collections.emptyMap()); + String ctid = null; + for (int i = 1; i <= columnCount; i++) { + final String columnName = metadata.getColumnName(i); + if (columnName.equalsIgnoreCase("ctid")) { + ctid = queryContext.getString(i); + continue; + } + + // convert to java types that will convert into reasonable json. + copyToJsonField(queryContext, i, jsonNode); + } + + assert Objects.nonNull(ctid); + return new RowDataWithCtid(jsonNode, ctid); + } + + public record RowDataWithCtid(JsonNode data, String ctid) { + + } +} diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index d781d6bcd2f70f..9d34ed32be59f7 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -14,6 +14,7 @@ import java.time.Instant; import java.time.OffsetDateTime; import java.util.Iterator; +import java.util.Objects; import java.util.function.BiFunction; import javax.annotation.CheckForNull; import org.apache.commons.lang3.StringUtils; @@ -26,7 +27,7 @@ public class CtidStateIterator extends AbstractIterator implemen public static final Integer SYNC_CHECKPOINT_RECORDS = 10_000; - private final Iterator messageIterator; + private final Iterator messageIterator; private final AirbyteStreamNameNamespacePair pair; private boolean hasEmittedFinalState; private String lastCtid; @@ -38,7 +39,7 @@ public class CtidStateIterator extends AbstractIterator implemen private final Duration syncCheckpointDuration; private final Long syncCheckpointRecords; - public CtidStateIterator(final Iterator messageIterator, + public CtidStateIterator(final Iterator messageIterator, final AirbyteStreamNameNamespacePair pair, final long relationFileNode, final JsonNode streamStateForIncrementalRun, @@ -59,6 +60,7 @@ public CtidStateIterator(final Iterator messageIterator, protected AirbyteMessage computeNext() { if (messageIterator.hasNext()) { if ((recordCount >= syncCheckpointRecords || Duration.between(lastCheckpoint, OffsetDateTime.now()).compareTo(syncCheckpointDuration) > 0) + && Objects.nonNull(lastCtid) && StringUtils.isNotBlank(lastCtid)) { final CtidStatus ctidStatus = new CtidStatus() .withVersion(CTID_STATUS_VERSION) @@ -73,12 +75,12 @@ protected AirbyteMessage computeNext() { } // Use try-catch to catch Exception that could occur when connection to the database fails try { - final AirbyteMessage message = messageIterator.next(); - if (message.getRecord().getData().hasNonNull("ctid")) { - this.lastCtid = message.getRecord().getData().get("ctid").asText(); + final AirbyteMessageWithCtid message = messageIterator.next(); + if (Objects.nonNull(message.ctid())) { + this.lastCtid = message.ctid(); } recordCount++; - return message; + return message.recordMessage(); } catch (final Exception e) { throw new RuntimeException(e); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index 8b21d2828e7c8a..d2859faa2ad855 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -3,13 +3,12 @@ import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; import com.fasterxml.jackson.databind.JsonNode; -import com.fasterxml.jackson.databind.node.ObjectNode; import io.airbyte.commons.stream.AirbyteStreamUtils; import io.airbyte.commons.util.AutoCloseableIterator; import io.airbyte.commons.util.AutoCloseableIterators; -import io.airbyte.db.JdbcCompatibleSourceOperations; import io.airbyte.db.jdbc.JdbcDatabase; import io.airbyte.integrations.source.postgres.PostgresType; +import io.airbyte.integrations.source.postgres.ctid.CtidPostgresSourceOperations.RowDataWithCtid; import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; import io.airbyte.integrations.source.relationaldb.DbSourceDiscoverUtil; import io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils; @@ -46,7 +45,7 @@ public class PostgresCtidHandler { private final JsonNode config; private final JdbcDatabase database; - private final JdbcCompatibleSourceOperations sourceOperations; + private final CtidPostgresSourceOperations sourceOperations; private final String quoteString; private final CtidStateManager ctidStateManager; private final Map fileNodes; @@ -55,7 +54,7 @@ public class PostgresCtidHandler { public PostgresCtidHandler(final JsonNode config, final JdbcDatabase database, - final JdbcCompatibleSourceOperations sourceOperations, + final CtidPostgresSourceOperations sourceOperations, final String quoteString, final Map fileNodes, final CtidStateManager ctidStateManager, @@ -94,18 +93,18 @@ public List> getIncrementalIterators( .map(CommonField::getName) .filter(CatalogHelpers.getTopLevelFieldNames(airbyteStream)::contains) .toList(); - final AutoCloseableIterator queryStream = queryTableCtid(selectedDatabaseFields, table.getNameSpace(), table.getName()); - final AutoCloseableIterator recordIterator = getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); + final AutoCloseableIterator queryStream = queryTableCtid(selectedDatabaseFields, table.getNameSpace(), table.getName()); + final AutoCloseableIterator recordIterator = getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); final AutoCloseableIterator recordAndMessageIterator = augmentWithState(recordIterator, pair); final AutoCloseableIterator logAugmented = augmentWithLogs(recordAndMessageIterator, pair, streamName); - iteratorList.add(swallowCtid(logAugmented, pair)); + iteratorList.add(logAugmented); } } return iteratorList; } - private AutoCloseableIterator queryTableCtid( + private AutoCloseableIterator queryTableCtid( final List columnNames, final String schemaName, final String tableName) { @@ -115,8 +114,8 @@ private AutoCloseableIterator queryTableCtid( AirbyteStreamUtils.convertFromNameAndNamespace(tableName, schemaName); return AutoCloseableIterators.lazyIterator(() -> { try { - final Stream stream = database.unsafeQuery( - connection -> createCtidQueryStatement(connection, columnNames, schemaName, tableName, airbyteStream), sourceOperations::rowToJson); + final Stream stream = database.unsafeQuery( + connection -> createCtidQueryStatement(connection, columnNames, schemaName, tableName, airbyteStream), sourceOperations::recordWithCtid); return AutoCloseableIterators.fromStream(stream, airbyteStream); } catch (final SQLException e) { throw new RuntimeException(e); @@ -145,7 +144,6 @@ private PreparedStatement createCtidQueryStatement( preparedStatement.setObject(1, "(0,0)"); } LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement); - preparedStatement.setQueryTimeout(120); return preparedStatement; } catch (final SQLException e) { throw new RuntimeException(e); @@ -153,18 +151,19 @@ private PreparedStatement createCtidQueryStatement( } // Transforms the given iterator to create an {@link AirbyteRecordMessage} - private static AutoCloseableIterator getRecordIterator( - final AutoCloseableIterator recordIterator, + private AutoCloseableIterator getRecordIterator( + final AutoCloseableIterator recordIterator, final String streamName, final String namespace, final long emittedAt) { - return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessage() + return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessageWithCtid(new AirbyteMessage() .withType(Type.RECORD) .withRecord(new AirbyteRecordMessage() .withStream(streamName) .withNamespace(namespace) .withEmittedAt(emittedAt) - .withData(r))); + .withData(r.data())), + r.ctid())); } // Augments the given iterator with record count logs. @@ -183,7 +182,7 @@ private AutoCloseableIterator augmentWithLogs(final AutoCloseabl }); } - private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, + private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, final AirbyteStreamNameNamespacePair pair) { final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(pair); @@ -199,21 +198,9 @@ private AutoCloseableIterator augmentWithState(final AutoCloseab final Long syncCheckpointRecords = config.get("sync_checkpoint_records") != null ? config.get("sync_checkpoint_records").asLong() : CtidStateIterator.SYNC_CHECKPOINT_RECORDS; - return AutoCloseableIterators.transform( - autoClosableIterator -> new CtidStateIterator(recordIterator, pair, latestFileNode, incrementalState, finalStateMessageSupplier, + return AutoCloseableIterators.transformIterator( + r -> new CtidStateIterator(r, pair, latestFileNode, incrementalState, finalStateMessageSupplier, syncCheckpointDuration, syncCheckpointRecords), recordIterator, pair); } - private AutoCloseableIterator swallowCtid(final AutoCloseableIterator iterator, - final AirbyteStreamNameNamespacePair pair) { - return AutoCloseableIterators.transform(iterator, - pair, - r -> { - if (r.getType() == Type.RECORD) { - ((ObjectNode) r.getRecord().getData()).remove("ctid"); - } - return r; - }); - } - } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index 4de29c06cbd823..a798e71bf032a1 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -33,11 +33,17 @@ public static StreamsCategorised categoriseStreams(final StateManager stateManag return; } final AirbyteStateMessage clonedState = Jsons.clone(s); - if (streamState.has("state_type") && streamState.get("state_type").asText().equalsIgnoreCase("ctid")) { - statesFromCtidSync.add(clonedState); - streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + if (streamState.has("state_type")) { + if (streamState.get("state_type").asText().equalsIgnoreCase("ctid")) { + statesFromCtidSync.add(clonedState); + streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); + } else if (streamState.get("state_type").asText().equalsIgnoreCase("xmin")) { + statesFromXminSync.add(clonedState); + } else { + throw new RuntimeException("Unknown state type: " + streamState.get("state_type").asText()); + } } else { - statesFromXminSync.add(clonedState); + throw new RuntimeException("State type not present"); } alreadySeenStreams.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); }); @@ -73,39 +79,19 @@ private static List identifyNewlyAddedStreams(final Con } - public static class StreamsCategorised { + public record StreamsCategorised(CtidStreams ctidStreams, + XminStreams xminStreams) { - public final CtidStreams ctidStreams; - public final XminStreams xminStreams; - - public StreamsCategorised(final CtidStreams ctidStreams, final XminStreams xminStreams) { - this.ctidStreams = ctidStreams; - this.xminStreams = xminStreams; - } } - public static class CtidStreams { - - public final List streamsForCtidSync; - public final List statesFromCtidSync; + public record CtidStreams(List streamsForCtidSync, + List statesFromCtidSync) { - public CtidStreams(final List streamsForCtidSync, - final List statesFromCtidSync) { - this.streamsForCtidSync = streamsForCtidSync; - this.statesFromCtidSync = statesFromCtidSync; - } } - public static class XminStreams { - - public final List streamsForXminSync; - public final List statesFromXminSync; + public record XminStreams(List streamsForXminSync, + List statesFromXminSync) { - public XminStreams(final List streamsForXminSync, - final List statesFromXminSync) { - this.streamsForXminSync = streamsForXminSync; - this.statesFromXminSync = statesFromXminSync; - } } } From 06f3ea72afc9a0f75757c7d97ced8132d23cb823 Mon Sep 17 00:00:00 2001 From: octavia-squidington-iii Date: Tue, 20 Jun 2023 09:05:34 +0000 Subject: [PATCH 18/27] =?UTF-8?q?=F0=9F=A4=96=20Auto=20format=20source-pos?= =?UTF-8?q?tgres=20code=20[skip=20ci]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../connectors/source-postgres/build.gradle | 1 + .../integration_tests/abnormal_state.json | 2 +- .../incremental_configured_catalog.json | 55 +- .../integration_tests/seed/basic.sql | 351 ++++++- .../integration_tests/seed/full.sql | 837 ++++++++++++++++- .../seed/full_without_nulls.sql | 879 +++++++++++++++++- .../source/postgres/PostgresQueryUtils.java | 7 +- .../source/postgres/PostgresSource.java | 5 +- .../postgres/ctid/AirbyteMessageWithCtid.java | 4 + .../ctid/CtidPostgresSourceOperations.java | 5 + .../postgres/ctid/CtidStateIterator.java | 19 +- .../postgres/ctid/CtidStateManager.java | 20 +- .../postgres/ctid/PostgresCtidHandler.java | 63 +- .../source/postgres/xmin/XminCtidUtils.java | 7 +- .../internal_models/internal_models.yaml | 2 +- .../postgres/XminPostgresSourceTest.java | 3 +- 16 files changed, 2120 insertions(+), 140 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index be730c71e81aad..ca362e246613e2 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -56,3 +56,4 @@ jsonSchema2Pojo { includeConstructors = false includeSetters = true } + diff --git a/airbyte-integrations/connectors/source-postgres/integration_tests/abnormal_state.json b/airbyte-integrations/connectors/source-postgres/integration_tests/abnormal_state.json index ec1eec43d6a7e9..c3e6b23a2b0db6 100644 --- a/airbyte-integrations/connectors/source-postgres/integration_tests/abnormal_state.json +++ b/airbyte-integrations/connectors/source-postgres/integration_tests/abnormal_state.json @@ -7,7 +7,7 @@ "stream_namespace": "public", "cursor_field": ["id"], "cursor": "4", - "cursor_record_count":1 + "cursor_record_count": 1 }, "stream_descriptor": { "name": "id_and_name", diff --git a/airbyte-integrations/connectors/source-postgres/integration_tests/incremental_configured_catalog.json b/airbyte-integrations/connectors/source-postgres/integration_tests/incremental_configured_catalog.json index fe8fa5fc2372da..648876fb50e1ab 100644 --- a/airbyte-integrations/connectors/source-postgres/integration_tests/incremental_configured_catalog.json +++ b/airbyte-integrations/connectors/source-postgres/integration_tests/incremental_configured_catalog.json @@ -1,32 +1,29 @@ { - "streams": [ - { - "stream": { - "name": "id_and_name", - "json_schema": { - "type": "object", - "properties": { - "name": { - "type": "string" - }, - "id": { - "type": "number", - "airbyte_type": "integer" - } - } - }, - "supported_sync_modes": [ - "full_refresh", - "incremental" - ], - "default_cursor_field": [], - "source_defined_primary_key": [], - "namespace": "public" + "streams": [ + { + "stream": { + "name": "id_and_name", + "json_schema": { + "type": "object", + "properties": { + "name": { + "type": "string" }, - "sync_mode": "incremental", - "destination_sync_mode": "append", - "cursor_field": ["id"], - "user_defined_primary_key": ["id"] - } - ] + "id": { + "type": "number", + "airbyte_type": "integer" + } + } + }, + "supported_sync_modes": ["full_refresh", "incremental"], + "default_cursor_field": [], + "source_defined_primary_key": [], + "namespace": "public" + }, + "sync_mode": "incremental", + "destination_sync_mode": "append", + "cursor_field": ["id"], + "user_defined_primary_key": ["id"] + } + ] } diff --git a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/basic.sql b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/basic.sql index 3adc7ca0b57acc..d7cb4fda899edb 100644 --- a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/basic.sql +++ b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/basic.sql @@ -1,17 +1,334 @@ -CREATE SCHEMA POSTGRES_BASIC; - -CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); -CREATE TYPE inventory_item AS (name text, supplier_id integer, price numeric); -SET lc_monetary TO 'en_US.utf8'; -SET TIMEZONE TO 'MST'; -CREATE EXTENSION hstore; - -CREATE TABLE POSTGRES_BASIC.TEST_DATASET(id INTEGER PRIMARY KEY, test_column_1 bigint,test_column_11 char,test_column_12 char(8),test_column_13 character,test_column_14 character(8),test_column_15 text,test_column_16 varchar,test_column_20 date not null default now(),test_column_21 date,test_column_23 float,test_column_24 double precision,test_column_27 int,test_column_28 integer,test_column_3 BIT(1),test_column_4 BIT(3),test_column_44 real,test_column_46 smallint,test_column_51 time without time zone,test_column_52 time,test_column_53 time without time zone not null default now(),test_column_54 timestamp,test_column_55 timestamp without time zone,test_column_56 timestamp without time zone default now(),test_column_57 timestamp,test_column_58 timestamp without time zone,test_column_59 timestamp without time zone not null default now(),test_column_60 timestamp with time zone,test_column_61 timestamptz,test_column_7 bool,test_column_70 time with time zone,test_column_71 timetz,test_column_8 boolean ); - -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (1, -9223372036854775808, 'a', '{asb123}', 'a', '{asb123}', 'a', 'a', '1999-01-08', '1999-01-08', '123', '123', 1001, 1001, B'0', B'101', 3.4145, -32768, '13:00:01', '13:00:01', '13:00:01', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', 0, 0, 0, TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', true, '13:00:01', '13:00:01', true); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (2, 9223372036854775807, '*', '{asb12}', '*', '{asb12}', 'abc', 'abc', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', -2147483648, -2147483648, B'0', B'101', 3.4145, 32767, '13:00:02+8', '13:00:02+8', '13:00:02+8', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', 0, 0, 0, TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', 'yes', '13:00:00+8', '13:00:00+8', 'yes'); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (3, 0, '*', '{asb12}', '*', '{asb12}', 'Миші йдуть на південь, не питай чому;', 'Миші йдуть на південь, не питай чому;', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', 2147483647, 2147483647, B'0', B'101', 3.4145, 32767, '13:00:03-8', '13:00:03-8', '13:00:03-8', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', 0, 0, 0, TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', '1', '13:00:03-8', '13:00:03-8', '1'); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (4, 0, '*', '{asb12}', '*', '{asb12}', '櫻花分店', '櫻花分店', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', 2147483647, 2147483647, B'0', B'101', 3.4145, 32767, '13:00:04Z', '13:00:04Z', '13:00:04Z', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', 0, 0, 0, TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', false, '13:00:04Z', '13:00:04Z', false); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (5, 0, '*', '{asb12}', '*', '{asb12}', '', '', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', 2147483647, 2147483647, B'0', B'101', 3.4145, 32767, '13:00:05.01234Z+8', '13:00:05.01234Z+8', '13:00:05.01234Z+8', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', 0, 0, 0, TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', 'no', '13:00:05.012345Z+8', '13:00:05.012345Z+8', 'no'); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (6, 0, '*', '{asb12}', '*', '{asb12}', '\xF0\x9F\x9A\x80', '\xF0\x9F\x9A\x80', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', 2147483647, 2147483647, B'0', B'101', 3.4145, 32767, '13:00:00Z-8', '13:00:00Z-8', '13:00:00Z-8', 'epoch', 'epoch', 'epoch', 0, 0, 0, TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', '0', '13:00:06.00000Z-8', '13:00:06.00000Z-8', '0'); -INSERT INTO POSTGRES_BASIC.TEST_DATASET VALUES (7, 0, '*', '{asb12}', '*', '{asb12}', '\xF0\x9F\x9A\x80', '\xF0\x9F\x9A\x80', '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', 2147483647, 2147483647, B'0', B'101', 3.4145, 32767, '24:00:00', '24:00:00', '24:00:00', 'epoch', 'epoch', 'epoch', 0, 0, 0, TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', '0', '13:00:06.00000Z-8', '13:00:06.00000Z-8', '0'); +CREATE + SCHEMA POSTGRES_BASIC; + +CREATE + TYPE mood AS ENUM( + 'sad', + 'ok', + 'happy' + ); + +CREATE + TYPE inventory_item AS( + name text, + supplier_id INTEGER, + price NUMERIC + ); +SET +lc_monetary TO 'en_US.utf8'; +SET +TIMEZONE TO 'MST'; + +CREATE + EXTENSION hstore; + +CREATE + TABLE + POSTGRES_BASIC.TEST_DATASET( + id INTEGER PRIMARY KEY, + test_column_1 BIGINT, + test_column_11 CHAR, + test_column_12 CHAR(8), + test_column_13 CHARACTER, + test_column_14 CHARACTER(8), + test_column_15 text, + test_column_16 VARCHAR, + test_column_20 DATE NOT NULL DEFAULT now(), + test_column_21 DATE, + test_column_23 FLOAT, + test_column_24 DOUBLE PRECISION, + test_column_27 INT, + test_column_28 INTEGER, + test_column_3 BIT(1), + test_column_4 BIT(3), + test_column_44 REAL, + test_column_46 SMALLINT, + test_column_51 TIME WITHOUT TIME ZONE, + test_column_52 TIME, + test_column_53 TIME WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_54 TIMESTAMP, + test_column_55 TIMESTAMP WITHOUT TIME ZONE, + test_column_56 TIMESTAMP WITHOUT TIME ZONE DEFAULT now(), + test_column_57 TIMESTAMP, + test_column_58 TIMESTAMP WITHOUT TIME ZONE, + test_column_59 TIMESTAMP WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_60 TIMESTAMP WITH TIME ZONE, + test_column_61 timestamptz, + test_column_7 bool, + test_column_70 TIME WITH TIME ZONE, + test_column_71 timetz, + test_column_8 BOOLEAN + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 1, + - 9223372036854775808, + 'a', + '{asb123}', + 'a', + '{asb123}', + 'a', + 'a', + '1999-01-08', + '1999-01-08', + '123', + '123', + 1001, + 1001, + B'0', + B'101', + 3.4145, + - 32768, + '13:00:01', + '13:00:01', + '13:00:01', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + TRUE, + '13:00:01', + '13:00:01', + TRUE + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 2, + 9223372036854775807, + '*', + '{asb12}', + '*', + '{asb12}', + 'abc', + 'abc', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + - 2147483648, + - 2147483648, + B'0', + B'101', + 3.4145, + 32767, + '13:00:02+8', + '13:00:02+8', + '13:00:02+8', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + 'yes', + '13:00:00+8', + '13:00:00+8', + 'yes' + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 3, + 0, + '*', + '{asb12}', + '*', + '{asb12}', + 'Миші йдуть на південь, не питай чому;', + 'Миші йдуть на південь, не питай чому;', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + 2147483647, + 2147483647, + B'0', + B'101', + 3.4145, + 32767, + '13:00:03-8', + '13:00:03-8', + '13:00:03-8', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + '1', + '13:00:03-8', + '13:00:03-8', + '1' + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 4, + 0, + '*', + '{asb12}', + '*', + '{asb12}', + '櫻花分店', + '櫻花分店', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + 2147483647, + 2147483647, + B'0', + B'101', + 3.4145, + 32767, + '13:00:04Z', + '13:00:04Z', + '13:00:04Z', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + FALSE, + '13:00:04Z', + '13:00:04Z', + FALSE + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 5, + 0, + '*', + '{asb12}', + '*', + '{asb12}', + '', + '', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + 2147483647, + 2147483647, + B'0', + B'101', + 3.4145, + 32767, + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + 'no', + '13:00:05.012345Z+8', + '13:00:05.012345Z+8', + 'no' + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 6, + 0, + '*', + '{asb12}', + '*', + '{asb12}', + '\xF0\x9F\x9A\x80', + '\xF0\x9F\x9A\x80', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + 2147483647, + 2147483647, + B'0', + B'101', + 3.4145, + 32767, + '13:00:00Z-8', + '13:00:00Z-8', + '13:00:00Z-8', + 'epoch', + 'epoch', + 'epoch', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + '0', + '13:00:06.00000Z-8', + '13:00:06.00000Z-8', + '0' + ); + +INSERT + INTO + POSTGRES_BASIC.TEST_DATASET + VALUES( + 7, + 0, + '*', + '{asb12}', + '*', + '{asb12}', + '\xF0\x9F\x9A\x80', + '\xF0\x9F\x9A\x80', + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + 2147483647, + 2147483647, + B'0', + B'101', + 3.4145, + 32767, + '24:00:00', + '24:00:00', + '24:00:00', + 'epoch', + 'epoch', + 'epoch', + 0, + 0, + 0, + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + '0', + '13:00:06.00000Z-8', + '13:00:06.00000Z-8', + '0' + ); diff --git a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full.sql b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full.sql index 7947393a298768..26edf7749a9fec 100644 --- a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full.sql +++ b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full.sql @@ -1,20 +1,825 @@ -CREATE SCHEMA POSTGRES_FULL; +CREATE + SCHEMA POSTGRES_FULL; -CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); -CREATE TYPE inventory_item AS (name text, supplier_id integer, price numeric); -SET lc_monetary TO 'en_US.utf8'; -SET TIMEZONE TO 'MST'; -CREATE EXTENSION hstore; +CREATE + TYPE mood AS ENUM( + 'sad', + 'ok', + 'happy' + ); -CREATE TABLE POSTGRES_FULL.TEST_DATASET(id INTEGER PRIMARY KEY, test_column_1 bigint,test_column_10 bytea,test_column_11 char,test_column_12 char(8),test_column_13 character,test_column_14 character(8),test_column_15 text,test_column_16 varchar,test_column_17 character varying(10),test_column_18 cidr,test_column_19 circle,test_column_2 bigserial,test_column_20 date not null default now(),test_column_21 date,test_column_22 float8,test_column_23 float,test_column_24 double precision,test_column_25 inet,test_column_26 int4,test_column_27 int,test_column_28 integer,test_column_29 interval,test_column_3 BIT(1),test_column_30 json,test_column_31 jsonb,test_column_32 line,test_column_33 lseg,test_column_34 macaddr,test_column_35 macaddr8,test_column_36 money,test_column_37 decimal,test_column_38 numeric,test_column_39 path,test_column_4 BIT(3),test_column_40 pg_lsn,test_column_41 point,test_column_42 polygon,test_column_43 float4,test_column_44 real,test_column_45 int2,test_column_46 smallint,test_column_47 serial2,test_column_48 smallserial,test_column_49 serial4,test_column_5 BIT VARYING(5),test_column_51 time without time zone,test_column_52 time,test_column_53 time without time zone not null default now(),test_column_54 timestamp,test_column_55 timestamp without time zone,test_column_56 timestamp without time zone default now(),test_column_57 timestamp,test_column_58 timestamp without time zone,test_column_59 timestamp without time zone not null default now(),test_column_6 BIT VARYING(5),test_column_60 timestamp with time zone,test_column_61 timestamptz,test_column_62 tsquery,test_column_63 tsvector,test_column_64 uuid,test_column_65 xml,test_column_66 mood,test_column_67 tsrange,test_column_68 inventory_item,test_column_69 hstore,test_column_7 bool,test_column_70 time with time zone,test_column_71 timetz,test_column_72 INT2[],test_column_73 INT4[],test_column_74 INT8[],test_column_75 OID[],test_column_76 VARCHAR[],test_column_77 CHAR(1)[],test_column_78 BPCHAR(2)[],test_column_79 TEXT[],test_column_8 boolean,test_column_80 NAME[],test_column_81 NUMERIC[],test_column_82 DECIMAL[],test_column_83 FLOAT4[],test_column_84 FLOAT8[],test_column_85 MONEY[],test_column_86 BOOL[],test_column_87 BIT[],test_column_88 BYTEA[],test_column_89 DATE[],test_column_9 box,test_column_90 TIME(6)[],test_column_91 TIMETZ[],test_column_92 TIMESTAMPTZ[],test_column_93 TIMESTAMP[] ); +CREATE + TYPE inventory_item AS( + name text, + supplier_id INTEGER, + price NUMERIC + ); +SET +lc_monetary TO 'en_US.utf8'; +SET +TIMEZONE TO 'MST'; -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (1, -9223372036854775808, null, 'a', '{asb123}', 'a', '{asb123}', 'a', 'a', '{asb123}', null, '(5,7),10', 1, '1999-01-08', '1999-01-08', '123', '123', '123', '198.24.10.0/24', null, null, null, null, B'0', null, null, '{4,5,6}', '((3,7),(15,18))', null, null, null, '123', '123', '((3,7),(15,18))', B'101', '7/A25801C8'::pg_lsn, '(3,7)', '((3,7),(15,18))', null, null, null, null, 1, 1, 1, B'101', '13:00:01', '13:00:01', '13:00:01', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', 'infinity', 'infinity', 'infinity', B'101', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', null, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', XMLPARSE (DOCUMENT 'Manual...'), 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +CREATE + EXTENSION hstore; + +CREATE + TABLE + POSTGRES_FULL.TEST_DATASET( + id INTEGER PRIMARY KEY, + test_column_1 BIGINT, + test_column_10 bytea, + test_column_11 CHAR, + test_column_12 CHAR(8), + test_column_13 CHARACTER, + test_column_14 CHARACTER(8), + test_column_15 text, + test_column_16 VARCHAR, + test_column_17 CHARACTER VARYING(10), + test_column_18 cidr, + test_column_19 circle, + test_column_2 bigserial, + test_column_20 DATE NOT NULL DEFAULT now(), + test_column_21 DATE, + test_column_22 float8, + test_column_23 FLOAT, + test_column_24 DOUBLE PRECISION, + test_column_25 inet, + test_column_26 int4, + test_column_27 INT, + test_column_28 INTEGER, + test_column_29 INTERVAL, + test_column_3 BIT(1), + test_column_30 json, + test_column_31 jsonb, + test_column_32 line, + test_column_33 lseg, + test_column_34 macaddr, + test_column_35 macaddr8, + test_column_36 money, + test_column_37 DECIMAL, + test_column_38 NUMERIC, + test_column_39 PATH, + test_column_4 BIT(3), + test_column_40 pg_lsn, + test_column_41 point, + test_column_42 polygon, + test_column_43 float4, + test_column_44 REAL, + test_column_45 int2, + test_column_46 SMALLINT, + test_column_47 serial2, + test_column_48 smallserial, + test_column_49 serial4, + test_column_5 BIT VARYING(5), + test_column_51 TIME WITHOUT TIME ZONE, + test_column_52 TIME, + test_column_53 TIME WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_54 TIMESTAMP, + test_column_55 TIMESTAMP WITHOUT TIME ZONE, + test_column_56 TIMESTAMP WITHOUT TIME ZONE DEFAULT now(), + test_column_57 TIMESTAMP, + test_column_58 TIMESTAMP WITHOUT TIME ZONE, + test_column_59 TIMESTAMP WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_6 BIT VARYING(5), + test_column_60 TIMESTAMP WITH TIME ZONE, + test_column_61 timestamptz, + test_column_62 tsquery, + test_column_63 tsvector, + test_column_64 uuid, + test_column_65 xml, + test_column_66 mood, + test_column_67 tsrange, + test_column_68 inventory_item, + test_column_69 hstore, + test_column_7 bool, + test_column_70 TIME WITH TIME ZONE, + test_column_71 timetz, + test_column_72 INT2 [], + test_column_73 INT4 [], + test_column_74 INT8 [], + test_column_75 OID [], + test_column_76 VARCHAR [], + test_column_77 CHAR(1)[], + test_column_78 BPCHAR(2)[], + test_column_79 TEXT [], + test_column_8 BOOLEAN, + test_column_80 NAME [], + test_column_81 NUMERIC [], + test_column_82 DECIMAL [], + test_column_83 FLOAT4 [], + test_column_84 FLOAT8 [], + test_column_85 MONEY [], + test_column_86 BOOL [], + test_column_87 BIT [], + test_column_88 BYTEA [], + test_column_89 DATE [], + test_column_9 box, + test_column_90 TIME(6)[], + test_column_91 TIMETZ [], + test_column_92 TIMESTAMPTZ [], + test_column_93 TIMESTAMP [] + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 1, + - 9223372036854775808, + NULL, + 'a', + '{asb123}', + 'a', + '{asb123}', + 'a', + 'a', + '{asb123}', + NULL, + '(5,7),10', + 1, + '1999-01-08', + '1999-01-08', + '123', + '123', + '123', + '198.24.10.0/24', + NULL, + NULL, + NULL, + NULL, + B'0', + NULL, + NULL, + '{4,5,6}', + '((3,7),(15,18))', + NULL, + NULL, + NULL, + '123', + '123', + '((3,7),(15,18))', + B'101', + '7/A25801C8'::pg_lsn, + '(3,7)', + '((3,7),(15,18))', + NULL, + NULL, + NULL, + NULL, + 1, + 1, + 1, + B'101', + '13:00:01', + '13:00:01', + '13:00:01', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + 'infinity', + 'infinity', + 'infinity', + B'101', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + NULL, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + XMLPARSE( + DOCUMENT 'Manual...' + ), + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, true, null, null, '{1,2,3}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', true, '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((3,7),(15,18))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (2, 9223372036854775807, decode('1234', 'hex'), '*', '{asb12}', '*', '{asb12}', 'abc', 'abc', '{asb12}', '192.168.100.128/25', '(0,0),0', 9223372036854775807, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.24.10.0', 1001, 1001, 1001, 'P1Y2M3DT4H5M6S', null, '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08:00:2b:01:02:03', '08:00:2b:01:02:03:04:05', '999.99', null, null, '((0,0),(0,0))', null, '0/0'::pg_lsn, '(0,0)', '((0,0),(0,0))', 3.4145, 3.4145, -32768, -32768, 32767, 32767, 2147483647, null, '13:00:02+8', '13:00:02+8', '13:00:02+8', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', '-infinity', '-infinity', '-infinity', null, TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', 'fat & (rat | cat)'::tsquery, null, null, null, null, null, null, null, 'yes', '13:00:01', '13:00:01', '{4,5,6}', null, null, null, null, null, null, null, 'yes', null, null, null, null, null, null, null, null, null, null, '((0,0),(0,0))', null, null, null, null); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (3, 0, '1234', null, null, null, null, 'Миші йдуть на південь, не питай чому;', 'Миші йдуть на південь, не питай чому;', null, '192.168/24', '(-10,-4),10', 0, null, null, null, null, null, '198.10/8', -2147483648, -2147483648, -2147483648, '-178000000', null, null, null, null, null, '08-00-2b-01-02-04', '08-00-2b-01-02-03-04-06', '1,001.01', '1234567890.1234567', '1234567890.1234567', null, null, null, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', null, null, 32767, 32767, 0, 0, 0, null, '13:00:03-8', '13:00:03-8', '13:00:03-8', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', null, null, null, null, TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', 'fat:ab & cat'::tsquery, null, null, '', null, null, null, null, '1', '13:00:00+8', '13:00:00+8', null, null, null, null, null, null, null, null, '1', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (4, null, 'abcd', null, null, null, null, '櫻花分店', '櫻花分店', null, '192.168.1', null, -9223372036854775808, null, null, null, null, null, null, 2147483647, 2147483647, 2147483647, '178000000', null, null, null, null, null, '08002b:010205', '08002b:0102030407', '-1,000', null, null, null, null, null, null, null, null, null, null, null, -32767, -32767, -2147483647, null, '13:00:04Z', '13:00:04Z', '13:00:04Z', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', null, null, null, null, TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', null, null, null, null, null, null, null, null, false, '13:00:03-8', '13:00:03-8', null, null, null, null, null, null, null, null, false, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (5, null, '\xabcd', null, null, null, null, '', '', null, '128.1', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '$999.99', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '13:00:05.01234Z+8', '13:00:05.01234Z+8', '13:00:05.01234Z+8', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', null, null, null, null, TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', null, null, null, null, null, null, null, null, 'no', '13:00:04Z', '13:00:04Z', null, null, null, null, null, null, null, null, 'no', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (6, null, null, null, null, null, null, null, null, null, '2001:4f8:3:ba::/64', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '$1001.01', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '13:00:00Z-8', '13:00:00Z-8', '13:00:00Z-8', 'epoch', 'epoch', 'epoch', null, null, null, null, null, null, null, null, null, null, null, null, null, null, '0', '13:00:05.012345Z+8', '13:00:05.012345Z+8', null, null, null, null, null, null, null, null, '0', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); -INSERT INTO POSTGRES_FULL.TEST_DATASET VALUES (7, null, null, null, null, null, null, '\xF0\x9F\x9A\x80', '\xF0\x9F\x9A\x80', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '-$1,000', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '24:00:00', '24:00:00', '24:00:00', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, '13:00:06.00000Z-8', '13:00:06.00000Z-8', null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null, null); +"weight" => "11.2 ounces"', + TRUE, + NULL, + NULL, + '{1,2,3}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + TRUE, + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((3,7),(15,18))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 2, + 9223372036854775807, + decode( + '1234', + 'hex' + ), + '*', + '{asb12}', + '*', + '{asb12}', + 'abc', + 'abc', + '{asb12}', + '192.168.100.128/25', + '(0,0),0', + 9223372036854775807, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.24.10.0', + 1001, + 1001, + 1001, + 'P1Y2M3DT4H5M6S', + NULL, + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08:00:2b:01:02:03', + '08:00:2b:01:02:03:04:05', + '999.99', + NULL, + NULL, + '((0,0),(0,0))', + NULL, + '0/0'::pg_lsn, + '(0,0)', + '((0,0),(0,0))', + 3.4145, + 3.4145, + - 32768, + - 32768, + 32767, + 32767, + 2147483647, + NULL, + '13:00:02+8', + '13:00:02+8', + '13:00:02+8', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + '-infinity', + '-infinity', + '-infinity', + NULL, + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + 'fat & (rat | cat)'::tsquery, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + 'yes', + '13:00:01', + '13:00:01', + '{4,5,6}', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + 'yes', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '((0,0),(0,0))', + NULL, + NULL, + NULL, + NULL + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 3, + 0, + '1234', + NULL, + NULL, + NULL, + NULL, + 'Миші йдуть на південь, не питай чому;', + 'Миші йдуть на південь, не питай чому;', + NULL, + '192.168/24', + '(-10,-4),10', + 0, + NULL, + NULL, + NULL, + NULL, + NULL, + '198.10/8', + - 2147483648, + - 2147483648, + - 2147483648, + '-178000000', + NULL, + NULL, + NULL, + NULL, + NULL, + '08-00-2b-01-02-04', + '08-00-2b-01-02-03-04-06', + '1,001.01', + '1234567890.1234567', + '1234567890.1234567', + NULL, + NULL, + NULL, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + NULL, + NULL, + 32767, + 32767, + 0, + 0, + 0, + NULL, + '13:00:03-8', + '13:00:03-8', + '13:00:03-8', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + NULL, + NULL, + NULL, + NULL, + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + 'fat:ab & cat'::tsquery, + NULL, + NULL, + '', + NULL, + NULL, + NULL, + NULL, + '1', + '13:00:00+8', + '13:00:00+8', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '1', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 4, + NULL, + 'abcd', + NULL, + NULL, + NULL, + NULL, + '櫻花分店', + '櫻花分店', + NULL, + '192.168.1', + NULL, + - 9223372036854775808, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + 2147483647, + 2147483647, + 2147483647, + '178000000', + NULL, + NULL, + NULL, + NULL, + NULL, + '08002b:010205', + '08002b:0102030407', + '-1,000', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + - 32767, + - 32767, + - 2147483647, + NULL, + '13:00:04Z', + '13:00:04Z', + '13:00:04Z', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + NULL, + NULL, + NULL, + NULL, + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + FALSE, + '13:00:03-8', + '13:00:03-8', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + FALSE, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 5, + NULL, + '\xabcd', + NULL, + NULL, + NULL, + NULL, + '', + '', + NULL, + '128.1', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '$999.99', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + NULL, + NULL, + NULL, + NULL, + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + 'no', + '13:00:04Z', + '13:00:04Z', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + 'no', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 6, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '2001:4f8:3:ba::/64', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '$1001.01', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '13:00:00Z-8', + '13:00:00Z-8', + '13:00:00Z-8', + 'epoch', + 'epoch', + 'epoch', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '0', + '13:00:05.012345Z+8', + '13:00:05.012345Z+8', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '0', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL + ); + +INSERT + INTO + POSTGRES_FULL.TEST_DATASET + VALUES( + 7, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '\xF0\x9F\x9A\x80', + '\xF0\x9F\x9A\x80', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '-$1,000', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '24:00:00', + '24:00:00', + '24:00:00', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + '13:00:06.00000Z-8', + '13:00:06.00000Z-8', + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL, + NULL + ); diff --git a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full_without_nulls.sql b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full_without_nulls.sql index 154b7c9fe63776..f101c9b7a3fa30 100644 --- a/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full_without_nulls.sql +++ b/airbyte-integrations/connectors/source-postgres/integration_tests/seed/full_without_nulls.sql @@ -1,38 +1,861 @@ -CREATE SCHEMA POSTGRES_FULL_NN; +CREATE + SCHEMA POSTGRES_FULL_NN; -CREATE TYPE mood AS ENUM ('sad', 'ok', 'happy'); -CREATE TYPE inventory_item AS (name text, supplier_id integer, price numeric); -SET lc_monetary TO 'en_US.utf8'; -SET TIMEZONE TO 'MST'; -CREATE EXTENSION hstore; +CREATE + TYPE mood AS ENUM( + 'sad', + 'ok', + 'happy' + ); -CREATE TABLE POSTGRES_FULL_NN.TEST_DATASET(id INTEGER PRIMARY KEY, test_column_1 bigint,test_column_10 bytea,test_column_11 char,test_column_12 char(8),test_column_13 character,test_column_14 character(8),test_column_15 text,test_column_16 varchar,test_column_17 character varying(10),test_column_18 cidr,test_column_19 circle,test_column_2 bigserial,test_column_20 date not null default now(),test_column_21 date,test_column_22 float8,test_column_23 float,test_column_24 double precision,test_column_25 inet,test_column_26 int4,test_column_27 int,test_column_28 integer,test_column_29 interval,test_column_3 BIT(1),test_column_30 json,test_column_31 jsonb,test_column_32 line,test_column_33 lseg,test_column_34 macaddr,test_column_35 macaddr8,test_column_36 money,test_column_37 decimal,test_column_38 numeric,test_column_39 path,test_column_4 BIT(3),test_column_40 pg_lsn,test_column_41 point,test_column_42 polygon,test_column_43 float4,test_column_44 real,test_column_45 int2,test_column_46 smallint,test_column_47 serial2,test_column_48 smallserial,test_column_49 serial4,test_column_5 BIT VARYING(5),test_column_51 time without time zone,test_column_52 time,test_column_53 time without time zone not null default now(),test_column_54 timestamp,test_column_55 timestamp without time zone,test_column_56 timestamp without time zone default now(),test_column_57 timestamp,test_column_58 timestamp without time zone,test_column_59 timestamp without time zone not null default now(),test_column_6 BIT VARYING(5),test_column_60 timestamp with time zone,test_column_61 timestamptz,test_column_62 tsquery,test_column_63 tsvector,test_column_64 uuid,test_column_65 xml,test_column_66 mood,test_column_67 tsrange,test_column_68 inventory_item,test_column_69 hstore,test_column_7 bool,test_column_70 time with time zone,test_column_71 timetz,test_column_72 INT2[],test_column_73 INT4[],test_column_74 INT8[],test_column_75 OID[],test_column_76 VARCHAR[],test_column_77 CHAR(1)[],test_column_78 BPCHAR(2)[],test_column_79 TEXT[],test_column_8 boolean,test_column_80 NAME[],test_column_81 NUMERIC[],test_column_82 DECIMAL[],test_column_83 FLOAT4[],test_column_84 FLOAT8[],test_column_85 MONEY[],test_column_86 BOOL[],test_column_87 BIT[],test_column_88 BYTEA[],test_column_89 DATE[],test_column_9 box,test_column_90 TIME(6)[],test_column_91 TIMETZ[],test_column_92 TIMESTAMPTZ[],test_column_93 TIMESTAMP[] ); +CREATE + TYPE inventory_item AS( + name text, + supplier_id INTEGER, + price NUMERIC + ); +SET +lc_monetary TO 'en_US.utf8'; +SET +TIMEZONE TO 'MST'; -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (1, -9223372036854775808, decode('1234', 'hex'), 'a', '{asb123}', 'a', '{asb123}', 'a', 'a', '{asb123}', '192.168.100.128/25', '(5,7),10', 1, '1999-01-08', '1999-01-08', '123', '123', '123', '198.24.10.0/24', 1001, 1001, 1001, 'P1Y2M3DT4H5M6S', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{4,5,6}', '((3,7),(15,18))', '08:00:2b:01:02:03', '08:00:2b:01:02:03:04:05', '999.99', '123', '123', '((3,7),(15,18))', B'101', '7/A25801C8'::pg_lsn, '(3,7)', '((3,7),(15,18))', 3.4145, 3.4145, -32768, -32768, 1, 1, 1, B'101', '13:00:01', '13:00:01', '13:00:01', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', TIMESTAMP '2004-10-19 10:23:00', 'infinity', 'infinity', 'infinity', B'101', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', 'fat & (rat | cat)'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', XMLPARSE (DOCUMENT 'Manual...'), 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +CREATE + EXTENSION hstore; + +CREATE + TABLE + POSTGRES_FULL_NN.TEST_DATASET( + id INTEGER PRIMARY KEY, + test_column_1 BIGINT, + test_column_10 bytea, + test_column_11 CHAR, + test_column_12 CHAR(8), + test_column_13 CHARACTER, + test_column_14 CHARACTER(8), + test_column_15 text, + test_column_16 VARCHAR, + test_column_17 CHARACTER VARYING(10), + test_column_18 cidr, + test_column_19 circle, + test_column_2 bigserial, + test_column_20 DATE NOT NULL DEFAULT now(), + test_column_21 DATE, + test_column_22 float8, + test_column_23 FLOAT, + test_column_24 DOUBLE PRECISION, + test_column_25 inet, + test_column_26 int4, + test_column_27 INT, + test_column_28 INTEGER, + test_column_29 INTERVAL, + test_column_3 BIT(1), + test_column_30 json, + test_column_31 jsonb, + test_column_32 line, + test_column_33 lseg, + test_column_34 macaddr, + test_column_35 macaddr8, + test_column_36 money, + test_column_37 DECIMAL, + test_column_38 NUMERIC, + test_column_39 PATH, + test_column_4 BIT(3), + test_column_40 pg_lsn, + test_column_41 point, + test_column_42 polygon, + test_column_43 float4, + test_column_44 REAL, + test_column_45 int2, + test_column_46 SMALLINT, + test_column_47 serial2, + test_column_48 smallserial, + test_column_49 serial4, + test_column_5 BIT VARYING(5), + test_column_51 TIME WITHOUT TIME ZONE, + test_column_52 TIME, + test_column_53 TIME WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_54 TIMESTAMP, + test_column_55 TIMESTAMP WITHOUT TIME ZONE, + test_column_56 TIMESTAMP WITHOUT TIME ZONE DEFAULT now(), + test_column_57 TIMESTAMP, + test_column_58 TIMESTAMP WITHOUT TIME ZONE, + test_column_59 TIMESTAMP WITHOUT TIME ZONE NOT NULL DEFAULT now(), + test_column_6 BIT VARYING(5), + test_column_60 TIMESTAMP WITH TIME ZONE, + test_column_61 timestamptz, + test_column_62 tsquery, + test_column_63 tsvector, + test_column_64 uuid, + test_column_65 xml, + test_column_66 mood, + test_column_67 tsrange, + test_column_68 inventory_item, + test_column_69 hstore, + test_column_7 bool, + test_column_70 TIME WITH TIME ZONE, + test_column_71 timetz, + test_column_72 INT2 [], + test_column_73 INT4 [], + test_column_74 INT8 [], + test_column_75 OID [], + test_column_76 VARCHAR [], + test_column_77 CHAR(1)[], + test_column_78 BPCHAR(2)[], + test_column_79 TEXT [], + test_column_8 BOOLEAN, + test_column_80 NAME [], + test_column_81 NUMERIC [], + test_column_82 DECIMAL [], + test_column_83 FLOAT4 [], + test_column_84 FLOAT8 [], + test_column_85 MONEY [], + test_column_86 BOOL [], + test_column_87 BIT [], + test_column_88 BYTEA [], + test_column_89 DATE [], + test_column_9 box, + test_column_90 TIME(6)[], + test_column_91 TIMETZ [], + test_column_92 TIMESTAMPTZ [], + test_column_93 TIMESTAMP [] + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 1, + - 9223372036854775808, + decode( + '1234', + 'hex' + ), + 'a', + '{asb123}', + 'a', + '{asb123}', + 'a', + 'a', + '{asb123}', + '192.168.100.128/25', + '(5,7),10', + 1, + '1999-01-08', + '1999-01-08', + '123', + '123', + '123', + '198.24.10.0/24', + 1001, + 1001, + 1001, + 'P1Y2M3DT4H5M6S', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{4,5,6}', + '((3,7),(15,18))', + '08:00:2b:01:02:03', + '08:00:2b:01:02:03:04:05', + '999.99', + '123', + '123', + '((3,7),(15,18))', + B'101', + '7/A25801C8'::pg_lsn, + '(3,7)', + '((3,7),(15,18))', + 3.4145, + 3.4145, + - 32768, + - 32768, + 1, + 1, + 1, + B'101', + '13:00:01', + '13:00:01', + '13:00:01', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + TIMESTAMP '2004-10-19 10:23:00', + 'infinity', + 'infinity', + 'infinity', + B'101', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:00-08', + 'fat & (rat | cat)'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + XMLPARSE( + DOCUMENT 'Manual...' + ), + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, true, '13:00:01', '13:00:01', '{1,2,3}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', true, '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((3,7),(15,18))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (2, 9223372036854775807, '1234', '*', '{asb12}', '*', '{asb12}', 'abc', 'abc', '{asb12}', '192.168/24', '(0,0),0', 9223372036854775807, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.24.10.0', -2147483648, -2147483648, -2147483648, '-178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08-00-2b-01-02-04', '08-00-2b-01-02-03-04-06', '1,001.01', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(0,0)', '((0,0),(0,0))', 3.4145, 3.4145, 32767, 32767, 32767, 32767, 2147483647, B'101', '13:00:02+8', '13:00:02+8', '13:00:02+8', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', TIMESTAMP '2004-10-19 10:23:54.123456', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + TRUE, + '13:00:01', + '13:00:01', + '{1,2,3}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + TRUE, + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((3,7),(15,18))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 2, + 9223372036854775807, + '1234', + '*', + '{asb12}', + '*', + '{asb12}', + 'abc', + 'abc', + '{asb12}', + '192.168/24', + '(0,0),0', + 9223372036854775807, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.24.10.0', + - 2147483648, + - 2147483648, + - 2147483648, + '-178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08-00-2b-01-02-04', + '08-00-2b-01-02-03-04-06', + '1,001.01', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(0,0)', + '((0,0),(0,0))', + 3.4145, + 3.4145, + 32767, + 32767, + 32767, + 32767, + 2147483647, + B'101', + '13:00:02+8', + '13:00:02+8', + '13:00:02+8', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + TIMESTAMP '2004-10-19 10:23:54.123456', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + TIMESTAMP WITH TIME ZONE '2004-10-19 10:23:54.123456-08', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, 'yes', '13:00:00+8', '13:00:00+8', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', 'yes', '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (3, 0, 'abcd', '*', '{asb12}', '*', '{asb12}', 'Миші йдуть на південь, не питай чому;', 'Миші йдуть на південь, не питай чому;', '{asb12}', '192.168.1', '(-10,-4),10', 0, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.10/8', 2147483647, 2147483647, 2147483647, '178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08002b:010205', '08002b:0102030407', '-1,000', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', 3.4145, 3.4145, 32767, 32767, 0, 0, 0, B'101', '13:00:03-8', '13:00:03-8', '13:00:03-8', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', TIMESTAMP '3004-10-19 10:23:54.123456 BC', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + 'yes', + '13:00:00+8', + '13:00:00+8', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + 'yes', + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 3, + 0, + 'abcd', + '*', + '{asb12}', + '*', + '{asb12}', + 'Миші йдуть на південь, не питай чому;', + 'Миші йдуть на південь, не питай чому;', + '{asb12}', + '192.168.1', + '(-10,-4),10', + 0, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.10/8', + 2147483647, + 2147483647, + 2147483647, + '178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08002b:010205', + '08002b:0102030407', + '-1,000', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + 3.4145, + 3.4145, + 32767, + 32767, + 0, + 0, + 0, + B'101', + '13:00:03-8', + '13:00:03-8', + '13:00:03-8', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + TIMESTAMP '3004-10-19 10:23:54.123456 BC', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + TIMESTAMP WITH TIME ZONE '3004-10-19 10:23:54.123456-08 BC', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, '1', '13:00:03-8', '13:00:03-8', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', '1', '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (4, 0, '\xabcd', '*', '{asb12}', '*', '{asb12}', '櫻花分店', '櫻花分店', '{asb12}', '128.1', '(-10,-4),10', -9223372036854775808, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.10/8', 2147483647, 2147483647, 2147483647, '178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08002b:010205', '08002b:0102030407', '$999.99', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', 3.4145, 3.4145, 32767, 32767, -32767, -32767, -2147483647, B'101', '13:00:04Z', '13:00:04Z', '13:00:04Z', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', TIMESTAMP '0001-01-01 00:00:00.000000', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + '1', + '13:00:03-8', + '13:00:03-8', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + '1', + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 4, + 0, + '\xabcd', + '*', + '{asb12}', + '*', + '{asb12}', + '櫻花分店', + '櫻花分店', + '{asb12}', + '128.1', + '(-10,-4),10', + - 9223372036854775808, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.10/8', + 2147483647, + 2147483647, + 2147483647, + '178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08002b:010205', + '08002b:0102030407', + '$999.99', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + 3.4145, + 3.4145, + 32767, + 32767, + - 32767, + - 32767, + - 2147483647, + B'101', + '13:00:04Z', + '13:00:04Z', + '13:00:04Z', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + TIMESTAMP '0001-01-01 00:00:00.000000', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 16:00:00.000000-08 BC', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, false, '13:00:04Z', '13:00:04Z', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', false, '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (5, 0, '\xabcd', '*', '{asb12}', '*', '{asb12}', '', '', '{asb12}', '2001:4f8:3:ba::/64', '(-10,-4),10', -9223372036854775808, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.10/8', 2147483647, 2147483647, 2147483647, '178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08002b:010205', '08002b:0102030407', '$1001.01', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', 3.4145, 3.4145, 32767, 32767, -32767, -32767, -2147483647, B'101', '13:00:05.01234Z+8', '13:00:05.01234Z+8', '13:00:05.01234Z+8', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', TIMESTAMP '0001-12-31 23:59:59.999999 BC', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + FALSE, + '13:00:04Z', + '13:00:04Z', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + FALSE, + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 5, + 0, + '\xabcd', + '*', + '{asb12}', + '*', + '{asb12}', + '', + '', + '{asb12}', + '2001:4f8:3:ba::/64', + '(-10,-4),10', + - 9223372036854775808, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.10/8', + 2147483647, + 2147483647, + 2147483647, + '178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08002b:010205', + '08002b:0102030407', + '$1001.01', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + 3.4145, + 3.4145, + 32767, + 32767, + - 32767, + - 32767, + - 2147483647, + B'101', + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + '13:00:05.01234Z+8', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + TIMESTAMP '0001-12-31 23:59:59.999999 BC', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, 'no', '13:00:05.012345Z+8', '13:00:05.012345Z+8', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', 'no', '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (6, 0, '\xabcd', '*', '{asb12}', '*', '{asb12}', '\xF0\x9F\x9A\x80', '\xF0\x9F\x9A\x80', '{asb12}', '2001:4f8:3:ba::/64', '(-10,-4),10', -9223372036854775808, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.10/8', 2147483647, 2147483647, 2147483647, '178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08002b:010205', '08002b:0102030407', '-$1,000', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', 3.4145, 3.4145, 32767, 32767, -32767, -32767, -2147483647, B'101', '13:00:00Z-8', '13:00:00Z-8', '13:00:00Z-8', 'epoch', 'epoch', 'epoch', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + 'no', + '13:00:05.012345Z+8', + '13:00:05.012345Z+8', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + 'no', + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 6, + 0, + '\xabcd', + '*', + '{asb12}', + '*', + '{asb12}', + '\xF0\x9F\x9A\x80', + '\xF0\x9F\x9A\x80', + '{asb12}', + '2001:4f8:3:ba::/64', + '(-10,-4),10', + - 9223372036854775808, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.10/8', + 2147483647, + 2147483647, + 2147483647, + '178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08002b:010205', + '08002b:0102030407', + '-$1,000', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + 3.4145, + 3.4145, + 32767, + 32767, + - 32767, + - 32767, + - 2147483647, + B'101', + '13:00:00Z-8', + '13:00:00Z-8', + '13:00:00Z-8', + 'epoch', + 'epoch', + 'epoch', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, '0', '13:00:06.00000Z-8', '13:00:06.00000Z-8', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', '0', '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); -INSERT INTO POSTGRES_FULL_NN.TEST_DATASET VALUES (7, 0, '\xabcd', '*', '{asb12}', '*', '{asb12}', '\xF0\x9F\x9A\x80', '\xF0\x9F\x9A\x80', '{asb12}', '2001:4f8:3:ba::/64', '(-10,-4),10', -9223372036854775808, '1991-02-10 BC', '1991-02-10 BC', '1234567890.1234567', '1234567890.1234567', '1234567890.1234567', '198.10/8', 2147483647, 2147483647, 2147483647, '178000000', B'0', '{"a": 10, "b": 15}', '[1, 2, 3]'::jsonb, '{0,1,0}', '((0,0),(0,0))', '08002b:010205', '08002b:0102030407', '-$1,000', '1234567890.1234567', '1234567890.1234567', '((0,0),(0,0))', B'101', '0/0'::pg_lsn, '(999999999999999999999999,0)', '((0,0),(999999999999999999999999,0))', 3.4145, 3.4145, 32767, 32767, -32767, -32767, -2147483647, B'101', '24:00:00', '24:00:00', '24:00:00', 'epoch', 'epoch', 'epoch', '-infinity', '-infinity', '-infinity', B'101', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', 'fat:ab & cat'::tsquery, to_tsvector('The quick brown fox jumped over the lazy dog.'), 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', '', 'happy', '(2010-01-01 14:30, 2010-01-01 15:30)', ROW('fuzzy dice', 42, 1.99), '"paperback" => "243","publisher" => "postgresqltutorial.com", +"weight" => "11.2 ounces"', + '0', + '13:00:06.00000Z-8', + '13:00:06.00000Z-8', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + '0', + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); + +INSERT + INTO + POSTGRES_FULL_NN.TEST_DATASET + VALUES( + 7, + 0, + '\xabcd', + '*', + '{asb12}', + '*', + '{asb12}', + '\xF0\x9F\x9A\x80', + '\xF0\x9F\x9A\x80', + '{asb12}', + '2001:4f8:3:ba::/64', + '(-10,-4),10', + - 9223372036854775808, + '1991-02-10 BC', + '1991-02-10 BC', + '1234567890.1234567', + '1234567890.1234567', + '1234567890.1234567', + '198.10/8', + 2147483647, + 2147483647, + 2147483647, + '178000000', + B'0', + '{"a": 10, "b": 15}', + '[1, 2, 3]'::jsonb, + '{0,1,0}', + '((0,0),(0,0))', + '08002b:010205', + '08002b:0102030407', + '-$1,000', + '1234567890.1234567', + '1234567890.1234567', + '((0,0),(0,0))', + B'101', + '0/0'::pg_lsn, + '(999999999999999999999999,0)', + '((0,0),(999999999999999999999999,0))', + 3.4145, + 3.4145, + 32767, + 32767, + - 32767, + - 32767, + - 2147483647, + B'101', + '24:00:00', + '24:00:00', + '24:00:00', + 'epoch', + 'epoch', + 'epoch', + '-infinity', + '-infinity', + '-infinity', + B'101', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + TIMESTAMP WITH TIME ZONE '0001-12-31 15:59:59.999999-08 BC', + 'fat:ab & cat'::tsquery, + to_tsvector('The quick brown fox jumped over the lazy dog.'), + 'a0eebc99-9c0b-4ef8-bb6d-6bb9bd380a11', + '', + 'happy', + '(2010-01-01 14:30, 2010-01-01 15:30)', + ROW( + 'fuzzy dice', + 42, + 1.99 + ), + '"paperback" => "243","publisher" => "postgresqltutorial.com", "language" => "English","ISBN-13" => "978-1449370000", -"weight" => "11.2 ounces"' -, '0', '13:00:06.00000Z-8', '13:00:06.00000Z-8', '{4,5,6}', '{-2147483648,2147483646}', '{-9223372036854775808,9223372036854775801}', '{564182,234181}', '{lorem ipsum,dolor sit,amet}', '{l,d,a}', '{l,d,a}', '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', '0', '{object,integer}', '{131070.23,231072.476596593}', '{131070.23,231072.476596593}', '{131070.237689,231072.476596593}', '{131070.237689,231072.476596593}', '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', '{true,yes,1,false,no,0,null}', '{null,1,0}', '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', '{1999-01-08,1991-02-10 BC}', '((0,0),(0,0))', '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}'); +"weight" => "11.2 ounces"', + '0', + '13:00:06.00000Z-8', + '13:00:06.00000Z-8', + '{4,5,6}', + '{-2147483648,2147483646}', + '{-9223372036854775808,9223372036854775801}', + '{564182,234181}', + '{lorem ipsum,dolor sit,amet}', + '{l,d,a}', + '{l,d,a}', + '{someeeeee loooooooooong teeeeext,vvvvvvveeeeeeeeeeeruyyyyyyyyy looooooooooooooooong teeeeeeeeeeeeeeext}', + '0', + '{object,integer}', + '{131070.23,231072.476596593}', + '{131070.23,231072.476596593}', + '{131070.237689,231072.476596593}', + '{131070.237689,231072.476596593}', + '{$999.99,$1001.01,45000, $1.001,$800,22222.006, 1001.01}', + '{true,yes,1,false,no,0,null}', + '{null,1,0}', + '{\xA6697E974E6A320F454390BE03F74955E8978F1A6971EA6730542E37B66179BC,\x4B52414B00000000000000000000000000000000000000000000000000000000}', + '{1999-01-08,1991-02-10 BC}', + '((0,0),(0,0))', + '{13:00:01,13:00:02+8,13:00:03-8,13:00:04Z,13:00:05.000000+8,13:00:00Z-8}', + '{null,13:00:01,13:00:00+8,13:00:03-8,13:00:04Z,13:00:05.012345Z+8,13:00:06.00000Z-8,13:00}', + '{null,2004-10-19 10:23:00-08,2004-10-19 10:23:54.123456-08}', + '{null,2004-10-19 10:23:00,2004-10-19 10:23:54.123456,3004-10-19 10:23:54.123456 BC}' + ); diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 7dbfc9400dcdfe..14c0906c22b8d2 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -99,10 +99,13 @@ public static XminStatus getXminStatus(final JdbcDatabase database) throws SQLEx .withStateType(StateType.XMIN); } - static Map fileNodeForStreams(final JdbcDatabase database, final List streams, final String quoteString) { + static Map fileNodeForStreams(final JdbcDatabase database, + final List streams, + final String quoteString) { final Map fileNodes = new HashMap<>(); streams.forEach(stream -> { - final AirbyteStreamNameNamespacePair namespacePair = new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()); + final AirbyteStreamNameNamespacePair namespacePair = + new AirbyteStreamNameNamespacePair(stream.getStream().getName(), stream.getStream().getNamespace()); final long l = fileNodeForStreams(database, namespacePair, quoteString); fileNodes.put(namespacePair, l); }); 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 e9424712ee659c..f66bbc57215543 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 @@ -20,7 +20,6 @@ import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.ROW_COUNT_RESULT_COL; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TABLE_ESTIMATE_QUERY; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TOTAL_BYTES_RESULT_COL; -import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.fileNodeForStreams; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.willVacuumingCauseIssue; import static io.airbyte.integrations.source.postgres.PostgresUtils.isIncrementalSyncMode; import static io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.categoriseStreams; @@ -469,7 +468,8 @@ public List> getIncrementalIterators(final if (!streamsCategorised.ctidStreams().streamsForCtidSync().isEmpty()) { if (willVacuumingCauseIssue(database, streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString())) { - //TODO : should we just skip these streams instead of failing the entire sync and just LOG warning stating that we are skipping these streams + // TODO : should we just skip these streams instead of failing the entire sync and just LOG warning + // stating that we are skipping these streams throw new RuntimeException("Stopping sync cause few streams are under Vacuuming, can not sync them using ctid"); } @@ -688,4 +688,5 @@ private List getFullTableEstimate(final JdbcDatabase database, Preconditions.checkState(jsonNodes.size() == 1); return jsonNodes; } + } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java index c961751580bdec..266975df720c5f 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.ctid; import io.airbyte.protocol.models.v0.AirbyteMessage; diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java index 64aa470274c4b3..591bb48fee1000 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.ctid; import com.fasterxml.jackson.databind.JsonNode; @@ -36,4 +40,5 @@ public RowDataWithCtid recordWithCtid(final ResultSet queryContext) throws SQLEx public record RowDataWithCtid(JsonNode data, String ctid) { } + } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 9d34ed32be59f7..8338928ca5feeb 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.ctid; import static io.airbyte.integrations.source.postgres.ctid.CtidStateManager.CTID_STATUS_VERSION; @@ -22,11 +26,11 @@ import org.slf4j.LoggerFactory; public class CtidStateIterator extends AbstractIterator implements Iterator { + private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateIterator.class); public static final Duration SYNC_CHECKPOINT_DURATION = Duration.ofMinutes(15); public static final Integer SYNC_CHECKPOINT_RECORDS = 10_000; - private final Iterator messageIterator; private final AirbyteStreamNameNamespacePair pair; private boolean hasEmittedFinalState; @@ -40,12 +44,12 @@ public class CtidStateIterator extends AbstractIterator implemen private final Long syncCheckpointRecords; public CtidStateIterator(final Iterator messageIterator, - final AirbyteStreamNameNamespacePair pair, - final long relationFileNode, - final JsonNode streamStateForIncrementalRun, - final BiFunction finalStateMessageSupplier, - final Duration checkpointDuration, - final Long checkpointRecords) { + final AirbyteStreamNameNamespacePair pair, + final long relationFileNode, + final JsonNode streamStateForIncrementalRun, + final BiFunction finalStateMessageSupplier, + final Duration checkpointDuration, + final Long checkpointRecords) { this.messageIterator = messageIterator; this.pair = pair; this.relationFileNode = relationFileNode; @@ -93,4 +97,5 @@ protected AirbyteMessage computeNext() { return endOfData(); } } + } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 63cdb04cb31276..9ff796605fdc57 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.ctid; import io.airbyte.commons.exceptions.ConfigErrorException; @@ -5,12 +9,12 @@ import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; import io.airbyte.protocol.models.AirbyteStreamNameNamespacePair; import io.airbyte.protocol.models.Jsons; -import io.airbyte.protocol.models.v0.AirbyteStateMessage; -import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; -import io.airbyte.protocol.models.v0.StreamDescriptor; import io.airbyte.protocol.models.v0.AirbyteMessage; import io.airbyte.protocol.models.v0.AirbyteMessage.Type; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; +import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.v0.AirbyteStreamState; +import io.airbyte.protocol.models.v0.StreamDescriptor; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -19,6 +23,7 @@ import org.slf4j.LoggerFactory; public class CtidStateManager { + private static final Logger LOGGER = LoggerFactory.getLogger(CtidStateManager.class); public static final long CTID_STATUS_VERSION = 2; private final Map pairToCtidStatus; @@ -31,7 +36,7 @@ public CtidStateManager(final List stateMessages, final Map } private static Map createPairToCtidStatusMap(final List stateMessages, - final Map fileNodes) { + final Map fileNodes) { final Map localMap = new HashMap<>(); if (stateMessages != null) { for (final AirbyteStateMessage stateMessage : stateMessages) { @@ -56,7 +61,9 @@ private static Map createPairToCtidS return localMap; } - private static boolean validateRelationFileNode(final CtidStatus ctidstatus, final AirbyteStreamNameNamespacePair pair, final Map fileNodes) { + private static boolean validateRelationFileNode(final CtidStatus ctidstatus, + final AirbyteStreamNameNamespacePair pair, + final Map fileNodes) { if (fileNodes.containsKey(pair)) { final Long fileNode = fileNodes.get(pair); return Objects.equals(ctidstatus.getRelationFilenode(), fileNode); @@ -68,7 +75,7 @@ public CtidStatus getCtidStatus(final AirbyteStreamNameNamespacePair pair) { return pairToCtidStatus.get(pair); } - //TODO : We will need a similar method to generate a GLOBAL state message for CDC + // TODO : We will need a similar method to generate a GLOBAL state message for CDC public static AirbyteMessage createPerStreamStateMessage(final AirbyteStreamNameNamespacePair pair, final CtidStatus ctidStatus) { final AirbyteStreamState airbyteStreamState = new AirbyteStreamState() @@ -87,4 +94,5 @@ public static AirbyteMessage createPerStreamStateMessage(final AirbyteStreamName .withType(Type.STATE) .withState(stateMessage); } + } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java index d2859faa2ad855..5b9410b2796fb0 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/PostgresCtidHandler.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.ctid; import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; @@ -53,13 +57,13 @@ public class PostgresCtidHandler { private final BiFunction finalStateMessageSupplier; public PostgresCtidHandler(final JsonNode config, - final JdbcDatabase database, - final CtidPostgresSourceOperations sourceOperations, - final String quoteString, - final Map fileNodes, - final CtidStateManager ctidStateManager, - final Function streamStateForIncrementalRunSupplier, - final BiFunction finalStateMessageSupplier) { + final JdbcDatabase database, + final CtidPostgresSourceOperations sourceOperations, + final String quoteString, + final Map fileNodes, + final CtidStateManager ctidStateManager, + final Function streamStateForIncrementalRunSupplier, + final BiFunction finalStateMessageSupplier) { this.config = config; this.database = database; this.sourceOperations = sourceOperations; @@ -69,10 +73,11 @@ public PostgresCtidHandler(final JsonNode config, this.streamStateForIncrementalRunSupplier = streamStateForIncrementalRunSupplier; this.finalStateMessageSupplier = finalStateMessageSupplier; } + public List> getIncrementalIterators( - final ConfiguredAirbyteCatalog catalog, - final Map>> tableNameToTable, - final Instant emmitedAt) { + final ConfiguredAirbyteCatalog catalog, + final Map>> tableNameToTable, + final Instant emmitedAt) { final List> iteratorList = new ArrayList<>(); for (final ConfiguredAirbyteStream airbyteStream : catalog.getStreams()) { final AirbyteStream stream = airbyteStream.getStream(); @@ -94,7 +99,8 @@ public List> getIncrementalIterators( .filter(CatalogHelpers.getTopLevelFieldNames(airbyteStream)::contains) .toList(); final AutoCloseableIterator queryStream = queryTableCtid(selectedDatabaseFields, table.getNameSpace(), table.getName()); - final AutoCloseableIterator recordIterator = getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); + final AutoCloseableIterator recordIterator = + getRecordIterator(queryStream, streamName, namespace, emmitedAt.toEpochMilli()); final AutoCloseableIterator recordAndMessageIterator = augmentWithState(recordIterator, pair); final AutoCloseableIterator logAugmented = augmentWithLogs(recordAndMessageIterator, pair, streamName); iteratorList.add(logAugmented); @@ -105,9 +111,9 @@ public List> getIncrementalIterators( } private AutoCloseableIterator queryTableCtid( - final List columnNames, - final String schemaName, - final String tableName) { + final List columnNames, + final String schemaName, + final String tableName) { LOGGER.info("Queueing query for table: {}", tableName); final AirbyteStreamNameNamespacePair airbyteStream = @@ -116,7 +122,7 @@ private AutoCloseableIterator queryTableCtid( try { final Stream stream = database.unsafeQuery( connection -> createCtidQueryStatement(connection, columnNames, schemaName, tableName, airbyteStream), sourceOperations::recordWithCtid); - return AutoCloseableIterators.fromStream(stream, airbyteStream); + return AutoCloseableIterators.fromStream(stream, airbyteStream); } catch (final SQLException e) { throw new RuntimeException(e); } @@ -124,11 +130,11 @@ private AutoCloseableIterator queryTableCtid( } private PreparedStatement createCtidQueryStatement( - final Connection connection, - final List columnNames, - final String schemaName, - final String tableName, - final AirbyteStreamNameNamespacePair airbyteStream) { + final Connection connection, + final List columnNames, + final String schemaName, + final String tableName, + final AirbyteStreamNameNamespacePair airbyteStream) { try { LOGGER.info("Preparing query for table: {}", tableName); final String fullTableName = getFullyQualifiedTableNameWithQuoting(schemaName, tableName, @@ -152,10 +158,10 @@ private PreparedStatement createCtidQueryStatement( // Transforms the given iterator to create an {@link AirbyteRecordMessage} private AutoCloseableIterator getRecordIterator( - final AutoCloseableIterator recordIterator, - final String streamName, - final String namespace, - final long emittedAt) { + final AutoCloseableIterator recordIterator, + final String streamName, + final String namespace, + final long emittedAt) { return AutoCloseableIterators.transform(recordIterator, r -> new AirbyteMessageWithCtid(new AirbyteMessage() .withType(Type.RECORD) .withRecord(new AirbyteRecordMessage() @@ -168,8 +174,8 @@ private AutoCloseableIterator getRecordIterator( // Augments the given iterator with record count logs. private AutoCloseableIterator augmentWithLogs(final AutoCloseableIterator iterator, - final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair, - final String streamName) { + final io.airbyte.protocol.models.AirbyteStreamNameNamespacePair pair, + final String streamName) { final AtomicLong recordCount = new AtomicLong(); return AutoCloseableIterators.transform(iterator, AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace()), @@ -183,7 +189,7 @@ private AutoCloseableIterator augmentWithLogs(final AutoCloseabl } private AutoCloseableIterator augmentWithState(final AutoCloseableIterator recordIterator, - final AirbyteStreamNameNamespacePair pair) { + final AirbyteStreamNameNamespacePair pair) { final CtidStatus currentCtidStatus = ctidStateManager.getCtidStatus(pair); final JsonNode incrementalState = @@ -200,7 +206,8 @@ private AutoCloseableIterator augmentWithState(final AutoCloseab return AutoCloseableIterators.transformIterator( r -> new CtidStateIterator(r, pair, latestFileNode, incrementalState, finalStateMessageSupplier, - syncCheckpointDuration, syncCheckpointRecords), recordIterator, pair); + syncCheckpointDuration, syncCheckpointRecords), + recordIterator, pair); } } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index a798e71bf032a1..5f1541cef3e4e7 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.source.postgres.xmin; import com.fasterxml.jackson.databind.JsonNode; @@ -67,7 +71,7 @@ public static StreamsCategorised categoriseStreams(final StateManager stateManag } private static List identifyNewlyAddedStreams(final ConfiguredAirbyteCatalog fullCatalog, - final Set alreadySeenStreams) { + final Set alreadySeenStreams) { final Set allStreams = AirbyteStreamNameNamespacePair.fromConfiguredCatalog(fullCatalog); final Set newlyAddedStreams = new HashSet<>(Sets.difference(allStreams, alreadySeenStreams)); @@ -78,7 +82,6 @@ private static List identifyNewlyAddedStreams(final Con .collect(Collectors.toList()); } - public record StreamsCategorised(CtidStreams ctidStreams, XminStreams xminStreams) { diff --git a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml index cd43c95bf60ead..a79f6ecda22fe7 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml +++ b/airbyte-integrations/connectors/source-postgres/src/main/resources/internal_models/internal_models.yaml @@ -50,4 +50,4 @@ definitions: type: object existingJavaType: com.fasterxml.jackson.databind.JsonNode relation_filenode: - type: integer \ No newline at end of file + type: integer diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java index 1760acb65cb9f3..d37f883fb52e27 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java @@ -214,7 +214,8 @@ void testReadSuccess() throws Exception { // Extract the state message and assert that it exists. It contains the xmin value, so validating // the actual value isn't useful right now. final List stateAfterFirstBatch = extractStateMessage(recordsFromFirstSync); - // We should have 3 state messages because we have set state emission frequency after each record in the test + // We should have 3 state messages because we have set state emission frequency after each record in + // the test assertEquals(3, stateAfterFirstBatch.size()); final AirbyteStateMessage firstStateMessage = stateAfterFirstBatch.get(0); From 643aff7e0ee35024faf412c7e3442dbab17ef36d Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 20 Jun 2023 14:54:03 +0530 Subject: [PATCH 19/27] skip streams under vacuum --- .../source/postgres/PostgresQueryUtils.java | 11 ++++++----- .../source/postgres/PostgresSource.java | 19 +++++++++++-------- .../source/postgres/xmin/XminCtidUtils.java | 7 +++++++ 3 files changed, 24 insertions(+), 13 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 14c0906c22b8d2..fae40c24fda584 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -130,8 +130,8 @@ public static long fileNodeForStreams(final JdbcDatabase database, final Airbyte } } - public static boolean willVacuumingCauseIssue(final JdbcDatabase database, final List streams, final String quoteString) { - final List streamsUnderVacuuming = new ArrayList<>(); + public static List streamsUnderVacuum(final JdbcDatabase database, final List streams, final String quoteString) { + final List streamsUnderVacuuming = new ArrayList<>(); streams.forEach(stream -> { final String streamName = stream.getStream().getName(); final String schemaName = stream.getStream().getNamespace(); @@ -143,15 +143,16 @@ public static boolean willVacuumingCauseIssue(final JdbcDatabase database, final resultSet -> JdbcUtils.getDefaultSourceOperations().rowToJson(resultSet)); if (jsonNodes.size() != 0) { Preconditions.checkState(jsonNodes.size() == 1); - LOGGER.warn("Full Vacuum currently in progress for table {} in {} phase", fullTableName, jsonNodes.get(0).get("phase")); - streamsUnderVacuuming.add(fullTableName); + LOGGER.warn("Full Vacuum currently in progress for table {} in {} phase, the table will be skipped from syncing data", fullTableName, + jsonNodes.get(0).get("phase")); + streamsUnderVacuuming.add(io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(stream)); } } catch (SQLException e) { // Assume it's safe to progress and skip relation node and vaccuum validation LOGGER.warn("Failed to fetch vacuum for table {} info. Going to move ahead with the sync assuming it's safe", fullTableName, e); } }); - return !streamsUnderVacuuming.isEmpty(); + return streamsUnderVacuuming; } } 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 f66bbc57215543..a6d2edbbca38db 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 @@ -20,7 +20,7 @@ import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.ROW_COUNT_RESULT_COL; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TABLE_ESTIMATE_QUERY; import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.TOTAL_BYTES_RESULT_COL; -import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.willVacuumingCauseIssue; +import static io.airbyte.integrations.source.postgres.PostgresQueryUtils.streamsUnderVacuum; import static io.airbyte.integrations.source.postgres.PostgresUtils.isIncrementalSyncMode; import static io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.categoriseStreams; import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; @@ -467,21 +467,24 @@ public List> getIncrementalIterators(final final List> xminIterator = new ArrayList<>(); if (!streamsCategorised.ctidStreams().streamsForCtidSync().isEmpty()) { - if (willVacuumingCauseIssue(database, streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString())) { - // TODO : should we just skip these streams instead of failing the entire sync and just LOG warning - // stating that we are skipping these streams - throw new RuntimeException("Stopping sync cause few streams are under Vacuuming, can not sync them using ctid"); - } + final List streamsUnderVacuum = streamsUnderVacuum(database, + streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString()); + final List finaListOfStreamsToBeSyncedViaCtid = + streamsUnderVacuum.isEmpty() ? streamsCategorised.ctidStreams().streamsForCtidSync() : + streamsCategorised.ctidStreams().streamsForCtidSync().stream() + .filter(c -> !streamsUnderVacuum.contains(AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(c))) + .toList(); final Map fileNodes = PostgresQueryUtils.fileNodeForStreams(database, - streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString()); + finaListOfStreamsToBeSyncedViaCtid, + getQuoteString()); final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams().statesFromCtidSync(), fileNodes); final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, new CtidPostgresSourceOperations(), getQuoteString(), fileNodes, ctidStateManager, namespacePair -> Jsons.jsonNode(xminStatus), (namespacePair, jsonState) -> XminStateManager.getAirbyteStateMessage(namespacePair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.ctidStreams().streamsForCtidSync()), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(finaListOfStreamsToBeSyncedViaCtid), tableNameToTable, emittedAt)); } if (!streamsCategorised.xminStreams().streamsForXminSync().isEmpty()) { diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index 5f1541cef3e4e7..210ed85a6286e8 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -19,6 +19,13 @@ import java.util.Set; import java.util.stream.Collectors; +/** + * The class mainly categorises the streams based on the state type into two categories : + * 1. Streams that need to be synced via ctid iterator: These + * are streams that are either newly added or did not complete their initial sync. + * 2. Streams that need to be synced via xmin iterator: These are + * streams that have completed their initial sync and are not syncing data incrementally. + */ public class XminCtidUtils { public static StreamsCategorised categoriseStreams(final StateManager stateManager, final ConfiguredAirbyteCatalog fullCatalog) { From 0756579f8a1064519ed3255af3323b2ac322aba0 Mon Sep 17 00:00:00 2001 From: octavia-squidington-iii Date: Tue, 20 Jun 2023 09:30:37 +0000 Subject: [PATCH 20/27] =?UTF-8?q?=F0=9F=A4=96=20Auto=20format=20source-pos?= =?UTF-8?q?tgres=20code=20[skip=20ci]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../connectors/source-postgres/build.gradle | 1 + .../integrations/source/postgres/PostgresQueryUtils.java | 4 +++- .../integrations/source/postgres/PostgresSource.java | 4 ++-- .../integrations/source/postgres/xmin/XminCtidUtils.java | 7 +++---- 4 files changed, 9 insertions(+), 7 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index ca362e246613e2..a28e207887e344 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -57,3 +57,4 @@ jsonSchema2Pojo { includeSetters = true } + diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index fae40c24fda584..68112ab6f50fea 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -130,7 +130,9 @@ public static long fileNodeForStreams(final JdbcDatabase database, final Airbyte } } - public static List streamsUnderVacuum(final JdbcDatabase database, final List streams, final String quoteString) { + public static List streamsUnderVacuum(final JdbcDatabase database, + final List streams, + final String quoteString) { final List streamsUnderVacuuming = new ArrayList<>(); streams.forEach(stream -> { final String streamName = stream.getStream().getName(); 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 a6d2edbbca38db..09c245e73058d2 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 @@ -471,8 +471,8 @@ public List> getIncrementalIterators(final streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString()); final List finaListOfStreamsToBeSyncedViaCtid = - streamsUnderVacuum.isEmpty() ? streamsCategorised.ctidStreams().streamsForCtidSync() : - streamsCategorised.ctidStreams().streamsForCtidSync().stream() + streamsUnderVacuum.isEmpty() ? streamsCategorised.ctidStreams().streamsForCtidSync() + : streamsCategorised.ctidStreams().streamsForCtidSync().stream() .filter(c -> !streamsUnderVacuum.contains(AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(c))) .toList(); final Map fileNodes = PostgresQueryUtils.fileNodeForStreams(database, diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index 210ed85a6286e8..b3f5054af038c9 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -20,10 +20,9 @@ import java.util.stream.Collectors; /** - * The class mainly categorises the streams based on the state type into two categories : - * 1. Streams that need to be synced via ctid iterator: These - * are streams that are either newly added or did not complete their initial sync. - * 2. Streams that need to be synced via xmin iterator: These are + * The class mainly categorises the streams based on the state type into two categories : 1. Streams + * that need to be synced via ctid iterator: These are streams that are either newly added or did + * not complete their initial sync. 2. Streams that need to be synced via xmin iterator: These are * streams that have completed their initial sync and are not syncing data incrementally. */ public class XminCtidUtils { From 134637c46a2084cf28a0add11f0a151622723923 Mon Sep 17 00:00:00 2001 From: subodh Date: Tue, 20 Jun 2023 15:02:10 +0530 Subject: [PATCH 21/27] update log message --- .../integrations/source/postgres/ctid/CtidStateIterator.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java index 8338928ca5feeb..4c4c440200aede 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateIterator.java @@ -90,9 +90,11 @@ protected AirbyteMessage computeNext() { } } else if (!hasEmittedFinalState) { hasEmittedFinalState = true; + final AirbyteStateMessage finalStateMessage = finalStateMessageSupplier.apply(pair, streamStateForIncrementalRun); + LOGGER.info("Emitting final state for stream {}, state is {}", pair, finalStateMessage.getStream().getStreamState()); return new AirbyteMessage() .withType(Type.STATE) - .withState(finalStateMessageSupplier.apply(pair, streamStateForIncrementalRun)); + .withState(finalStateMessage); } else { return endOfData(); } From 1e57a698c532c5f638472458aaa9627ec8e78fa4 Mon Sep 17 00:00:00 2001 From: octavia-squidington-iii Date: Tue, 20 Jun 2023 10:07:40 +0000 Subject: [PATCH 22/27] =?UTF-8?q?=F0=9F=A4=96=20Auto=20format=20source-pos?= =?UTF-8?q?tgres=20code=20[skip=20ci]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- airbyte-integrations/connectors/source-postgres/build.gradle | 1 + 1 file changed, 1 insertion(+) diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index a28e207887e344..63b9ead02d35be 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -58,3 +58,4 @@ jsonSchema2Pojo { } + From f7dddd846be8363eeaa0308c5d341ccc45a86f93 Mon Sep 17 00:00:00 2001 From: Rodi Reich Zilberman <867491+rodireich@users.noreply.github.com> Date: Tue, 20 Jun 2023 14:53:44 -0700 Subject: [PATCH 23/27] comment --- .../source/postgres/ctid/AirbyteMessageWithCtid.java | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java index 266975df720c5f..3e9d16a5f50382 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java @@ -6,6 +6,14 @@ import io.airbyte.protocol.models.v0.AirbyteMessage; +/** + * ctid of rows is queried as part of our sync and is used to checkpoint + * to be able to restart failed sync from a known last point. + * Since we never want to emit a ctid it is kept in a different field, + * to save us an expensive JsonNode.remove() operation. + * @param recordMessage row fields to emit + * @param ctid ctid + */ public record AirbyteMessageWithCtid(AirbyteMessage recordMessage, String ctid) { } From 854b414f7905d2b8f3b3f086e5574282e83eca99 Mon Sep 17 00:00:00 2001 From: subodh Date: Wed, 21 Jun 2023 20:47:12 +0530 Subject: [PATCH 24/27] latest round of review comments --- .../source/postgres/PostgresQueryUtils.java | 3 +- .../source/postgres/PostgresSource.java | 14 ++- .../postgres/ctid/CtidStateManager.java | 4 + .../source/postgres/xmin/XminCtidUtils.java | 12 +-- .../postgres/xmin/XminStateManager.java | 1 + .../postgres/XminPostgresSourceTest.java | 18 +--- .../postgres/xmin/XminCtidUtilsTest.java | 100 ++++++++++++++++++ 7 files changed, 123 insertions(+), 29 deletions(-) create mode 100644 airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java index 68112ab6f50fea..03ed5d942d48ce 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/PostgresQueryUtils.java @@ -4,6 +4,7 @@ package io.airbyte.integrations.source.postgres; +import static io.airbyte.integrations.source.postgres.xmin.XminStateManager.XMIN_STATE_VERSION; import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; import com.fasterxml.jackson.databind.JsonNode; @@ -95,7 +96,7 @@ public static XminStatus getXminStatus(final JdbcDatabase database) throws SQLEx .withNumWraparound(result.get(NUM_WRAPAROUND_COL).asLong()) .withXminXidValue(result.get(XMIN_XID_VALUE_COL).asLong()) .withXminRawValue(result.get(XMIN_RAW_VALUE_COL).asLong()) - .withVersion(2L) + .withVersion(XMIN_STATE_VERSION) .withStateType(StateType.XMIN); } 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 09c245e73058d2..a809dc8fddce45 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 @@ -460,8 +460,6 @@ public List> getIncrementalIterators(final } else if (PostgresUtils.isXmin(sourceConfig) && isIncrementalSyncMode(catalog)) { final StreamsCategorised streamsCategorised = categoriseStreams(stateManager, catalog); - LOGGER.info("Streams to be synced via ctid : {}", streamsCategorised.ctidStreams().streamsForCtidSync().size()); - LOGGER.info("Streams to be synced via xmin : {}", streamsCategorised.xminStreams().streamsForXminSync().size()); final List> ctidIterator = new ArrayList<>(); final List> xminIterator = new ArrayList<>(); @@ -470,13 +468,14 @@ public List> getIncrementalIterators(final final List streamsUnderVacuum = streamsUnderVacuum(database, streamsCategorised.ctidStreams().streamsForCtidSync(), getQuoteString()); - final List finaListOfStreamsToBeSyncedViaCtid = + final List finalListOfStreamsToBeSyncedViaCtid = streamsUnderVacuum.isEmpty() ? streamsCategorised.ctidStreams().streamsForCtidSync() : streamsCategorised.ctidStreams().streamsForCtidSync().stream() .filter(c -> !streamsUnderVacuum.contains(AirbyteStreamNameNamespacePair.fromConfiguredAirbyteSteam(c))) .toList(); + LOGGER.info("Streams to be synced via ctid : {}", finalListOfStreamsToBeSyncedViaCtid.size()); final Map fileNodes = PostgresQueryUtils.fileNodeForStreams(database, - finaListOfStreamsToBeSyncedViaCtid, + finalListOfStreamsToBeSyncedViaCtid, getQuoteString()); final CtidStateManager ctidStateManager = new CtidStateManager(streamsCategorised.ctidStreams().statesFromCtidSync(), fileNodes); final PostgresCtidHandler ctidHandler = new PostgresCtidHandler(sourceConfig, database, new CtidPostgresSourceOperations(), getQuoteString(), @@ -484,15 +483,20 @@ public List> getIncrementalIterators(final namespacePair -> Jsons.jsonNode(xminStatus), (namespacePair, jsonState) -> XminStateManager.getAirbyteStateMessage(namespacePair, Jsons.object(jsonState, XminStatus.class))); ctidIterator.addAll(ctidHandler.getIncrementalIterators( - new ConfiguredAirbyteCatalog().withStreams(finaListOfStreamsToBeSyncedViaCtid), tableNameToTable, emittedAt)); + new ConfiguredAirbyteCatalog().withStreams(finalListOfStreamsToBeSyncedViaCtid), tableNameToTable, emittedAt)); + } else { + LOGGER.info("No Streams will be synced via ctid."); } if (!streamsCategorised.xminStreams().streamsForXminSync().isEmpty()) { + LOGGER.info("Streams to be synced via xmin : {}", streamsCategorised.xminStreams().streamsForXminSync().size()); final XminStateManager xminStateManager = new XminStateManager(streamsCategorised.xminStreams().statesFromXminSync()); final PostgresXminHandler xminHandler = new PostgresXminHandler(database, sourceOperations, getQuoteString(), xminStatus, xminStateManager); xminIterator.addAll(xminHandler.getIncrementalIterators( new ConfiguredAirbyteCatalog().withStreams(streamsCategorised.xminStreams().streamsForXminSync()), tableNameToTable, emittedAt)); + } else { + LOGGER.info("No Streams will be synced via xmin."); } return Stream diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java index 9ff796605fdc57..c6aecfbd71aba5 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidStateManager.java @@ -54,6 +54,10 @@ private static Map createPairToCtidS } if (validateRelationFileNode(ctidStatus, pair, fileNodes)) { localMap.put(pair, ctidStatus); + } else { + LOGGER.warn( + "The relation file node for table in source db {} is not equal to the saved ctid state, a full sync from scratch will be triggered.", + pair); } } } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index b3f5054af038c9..6ff588fa2dfbf2 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -36,19 +36,19 @@ public static StreamsCategorised categoriseStreams(final StateManager stateManag final Set streamsStillInCtidSync = new HashSet<>(); if (rawStateMessages != null) { - rawStateMessages.forEach(s -> { - final JsonNode streamState = s.getStream().getStreamState(); - final StreamDescriptor streamDescriptor = s.getStream().getStreamDescriptor(); + rawStateMessages.forEach(stateMessage -> { + final JsonNode streamState = stateMessage.getStream().getStreamState(); + final StreamDescriptor streamDescriptor = stateMessage.getStream().getStreamDescriptor(); if (streamState == null || streamDescriptor == null) { return; } - final AirbyteStateMessage clonedState = Jsons.clone(s); + if (streamState.has("state_type")) { if (streamState.get("state_type").asText().equalsIgnoreCase("ctid")) { - statesFromCtidSync.add(clonedState); + statesFromCtidSync.add(stateMessage); streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); } else if (streamState.get("state_type").asText().equalsIgnoreCase("xmin")) { - statesFromXminSync.add(clonedState); + statesFromXminSync.add(stateMessage); } else { throw new RuntimeException("Unknown state type: " + streamState.get("state_type").asText()); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java index a883a11d5469cc..14cd2cef09568a 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminStateManager.java @@ -26,6 +26,7 @@ public class XminStateManager { private static final Logger LOGGER = LoggerFactory.getLogger(XminStateManager.class); + public static final long XMIN_STATE_VERSION = 2L; private final Map pairToXminStatus; diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java index d37f883fb52e27..f87edae0bf2da8 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/XminPostgresSourceTest.java @@ -271,7 +271,7 @@ void testReadSuccess() throws Exception { final List stateAfterSyncWithCtidState = extractStateMessage(recordsFromSyncRunningWithACtidState); // Since only 2 records should be emitted so 2 state messages are expected assertEquals(2, stateAfterSyncWithCtidState.size()); - assertEqualsCtidState(secondStateMessage.getStream().getStreamState(), stateAfterSyncWithCtidState.get(0).getStream().getStreamState()); + assertEquals(secondStateMessage, stateAfterSyncWithCtidState.get(0)); assertEquals(thirdStateMessage, stateAfterSyncWithCtidState.get(1)); assertMessageSequence(recordsFromSyncRunningWithACtidState); @@ -319,22 +319,6 @@ void testReadSuccess() throws Exception { .get("xmin_raw_value").asLong()); } - private void assertEqualsCtidState(final JsonNode left, final JsonNode right) { - assertEquals(left.get("version").asText(), right.get("version").asText()); - assertEquals(left.get("state_type").asText(), right.get("state_type").asText()); - assertEquals(left.get("ctid").asText(), right.get("ctid").asText()); - assertEqualsXminState(left.get("incremental_state"), right.get("incremental_state")); - assertEquals(left.get("relation_filenode"), right.get("relation_filenode")); - } - - private void assertEqualsXminState(final JsonNode left, final JsonNode right) { - assertEquals(left.get("version").asText(), right.get("version").asText()); - assertEquals(left.get("state_type").asText(), right.get("state_type").asText()); - assertEquals(left.get("num_wraparound").asText(), right.get("num_wraparound").asText()); - assertEquals(left.get("xmin_xid_value").asText(), right.get("xmin_xid_value").asText()); - assertEquals(left.get("xmin_raw_value").asText(), right.get("xmin_raw_value").asText()); - } - // Assert that the state message is the last message to be emitted. private static void assertMessageSequence(final List messages) { assertEquals(Type.STATE, messages.get(messages.size() - 1).getType()); diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java new file mode 100644 index 00000000000000..24f2fabebe0af8 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java @@ -0,0 +1,100 @@ +package io.airbyte.integrations.source.postgres.xmin; + + +import static org.assertj.core.api.Assertions.assertThat; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.collect.Lists; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.postgres.internal.models.CtidStatus; +import io.airbyte.integrations.source.postgres.internal.models.InternalModels.StateType; +import io.airbyte.integrations.source.postgres.internal.models.XminStatus; +import io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.StreamsCategorised; +import io.airbyte.integrations.source.relationaldb.state.StreamStateManager; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; +import io.airbyte.protocol.models.Field; +import io.airbyte.protocol.models.JsonSchemaType; +import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; +import io.airbyte.protocol.models.v0.AirbyteStreamState; +import io.airbyte.protocol.models.v0.CatalogHelpers; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog; +import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream; +import io.airbyte.protocol.models.v0.StreamDescriptor; +import io.airbyte.protocol.models.v0.SyncMode; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import org.junit.jupiter.api.Test; + +public class XminCtidUtilsTest { + + private static final ConfiguredAirbyteStream MODELS_STREAM = CatalogHelpers.toDefaultConfiguredStream(CatalogHelpers.createAirbyteStream( + "MODELS_STREAM_NAME", + "MODELS_SCHEMA", + Field.of("COL_ID", JsonSchemaType.INTEGER), + Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), + Field.of("COL_MODEL", JsonSchemaType.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) + .withSourceDefinedPrimaryKey(List.of(List.of("COL_ID")))); + + private static final ConfiguredAirbyteStream MODELS_STREAM_2 = CatalogHelpers.toDefaultConfiguredStream(CatalogHelpers.createAirbyteStream( + "MODELS_STREAM_NAME_2", + "MODELS_SCHEMA", + Field.of("COL_ID", JsonSchemaType.INTEGER), + Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), + Field.of("COL_MODEL", JsonSchemaType.STRING)) + .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) + .withSourceDefinedPrimaryKey(List.of(List.of("COL_ID")))); + + @Test + public void emptyStateTest() { + final ConfiguredAirbyteCatalog configuredCatalog = new ConfiguredAirbyteCatalog().withStreams(Arrays.asList(MODELS_STREAM, MODELS_STREAM_2)); + + final StreamStateManager streamStateManager = new StreamStateManager(Collections.emptyList(), configuredCatalog); + final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog); + + assertTrue(streamsCategorised.xminStreams().streamsForXminSync().isEmpty()); + assertTrue(streamsCategorised.xminStreams().statesFromXminSync().isEmpty()); + + assertEquals(2, streamsCategorised.ctidStreams().streamsForCtidSync().size()); + assertThat(streamsCategorised.ctidStreams().streamsForCtidSync()).containsExactlyInAnyOrder(MODELS_STREAM, MODELS_STREAM_2); + assertTrue(streamsCategorised.ctidStreams().statesFromCtidSync().isEmpty()); + } + + @Test + public void correctCategorisationTest() { + final ConfiguredAirbyteCatalog configuredCatalog = new ConfiguredAirbyteCatalog().withStreams(Arrays.asList(MODELS_STREAM, MODELS_STREAM_2)); + final XminStatus xminStatus = new XminStatus().withStateType(StateType.XMIN).withVersion(2L).withXminXidValue(9L).withXminRawValue(9L) + .withNumWraparound(1L); + final JsonNode xminStatusAsJson = Jsons.jsonNode(xminStatus); + final AirbyteStateMessage xminState = generateStateMessage(xminStatusAsJson, + new StreamDescriptor().withName(MODELS_STREAM.getStream().getName()).withNamespace(MODELS_STREAM.getStream().getNamespace())); + + final CtidStatus ctidStatus = new CtidStatus().withStateType(StateType.CTID).withVersion(2L).withCtid("123").withRelationFilenode(456L) + .withIncrementalState(xminStatusAsJson); + final JsonNode ctidStatusAsJson = Jsons.jsonNode(ctidStatus); + final AirbyteStateMessage ctidState = generateStateMessage(ctidStatusAsJson, + new StreamDescriptor().withName(MODELS_STREAM_2.getStream().getName()).withNamespace(MODELS_STREAM_2.getStream().getNamespace())); + + final StreamStateManager streamStateManager = new StreamStateManager(Arrays.asList(xminState, ctidState), configuredCatalog); + final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog); + + assertEquals(1, streamsCategorised.xminStreams().streamsForXminSync().size()); + assertEquals(MODELS_STREAM, streamsCategorised.xminStreams().streamsForXminSync().get(0)); + assertEquals(1, streamsCategorised.xminStreams().statesFromXminSync().size()); + assertEquals(xminState, streamsCategorised.xminStreams().statesFromXminSync().get(0)); + + assertEquals(1, streamsCategorised.ctidStreams().streamsForCtidSync().size()); + assertEquals(MODELS_STREAM_2, streamsCategorised.ctidStreams().streamsForCtidSync().get(0)); + assertEquals(1, streamsCategorised.ctidStreams().statesFromCtidSync().size()); + assertEquals(ctidState, streamsCategorised.ctidStreams().statesFromCtidSync().get(0)); + } + + private AirbyteStateMessage generateStateMessage(final JsonNode stateData, final StreamDescriptor streamDescriptor) { + return new AirbyteStateMessage().withType(AirbyteStateType.STREAM) + .withStream(new AirbyteStreamState().withStreamDescriptor(streamDescriptor).withStreamState(stateData)); + } + +} From 65f4711f47232da715ac6d0cec7b31f51f9044db Mon Sep 17 00:00:00 2001 From: subodh Date: Wed, 21 Jun 2023 20:47:30 +0530 Subject: [PATCH 25/27] missed this file --- .../source/postgres/ctid/CtidPostgresSourceOperations.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java index 591bb48fee1000..62047de57e23d3 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/CtidPostgresSourceOperations.java @@ -16,6 +16,8 @@ public class CtidPostgresSourceOperations extends PostgresSourceOperations { + private static final String CTID = "ctid"; + public RowDataWithCtid recordWithCtid(final ResultSet queryContext) throws SQLException { // the first call communicates with the database. after that the result is cached. final ResultSetMetaData metadata = queryContext.getMetaData(); @@ -24,7 +26,7 @@ public RowDataWithCtid recordWithCtid(final ResultSet queryContext) throws SQLEx String ctid = null; for (int i = 1; i <= columnCount; i++) { final String columnName = metadata.getColumnName(i); - if (columnName.equalsIgnoreCase("ctid")) { + if (columnName.equalsIgnoreCase(CTID)) { ctid = queryContext.getString(i); continue; } From 52eb61de163fcb090a56cf73ea072ddfdcc82f8a Mon Sep 17 00:00:00 2001 From: octavia-squidington-iii Date: Wed, 21 Jun 2023 15:21:12 +0000 Subject: [PATCH 26/27] =?UTF-8?q?=F0=9F=A4=96=20Auto=20format=20source-pos?= =?UTF-8?q?tgres=20code=20[skip=20ci]?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- .../connectors/source-postgres/build.gradle | 1 + .../postgres/ctid/AirbyteMessageWithCtid.java | 8 +++--- .../postgres/xmin/XminCtidUtilsTest.java | 27 ++++++++++--------- 3 files changed, 20 insertions(+), 16 deletions(-) diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index ca362e246613e2..a28e207887e344 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -57,3 +57,4 @@ jsonSchema2Pojo { includeSetters = true } + diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java index 3e9d16a5f50382..cbadc1af7044df 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/ctid/AirbyteMessageWithCtid.java @@ -7,10 +7,10 @@ import io.airbyte.protocol.models.v0.AirbyteMessage; /** - * ctid of rows is queried as part of our sync and is used to checkpoint - * to be able to restart failed sync from a known last point. - * Since we never want to emit a ctid it is kept in a different field, - * to save us an expensive JsonNode.remove() operation. + * ctid of rows is queried as part of our sync and is used to checkpoint to be able to restart + * failed sync from a known last point. Since we never want to emit a ctid it is kept in a different + * field, to save us an expensive JsonNode.remove() operation. + * * @param recordMessage row fields to emit * @param ctid ctid */ diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java index 24f2fabebe0af8..6e34678023e31b 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java @@ -1,5 +1,8 @@ -package io.airbyte.integrations.source.postgres.xmin; +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ +package io.airbyte.integrations.source.postgres.xmin; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -13,9 +16,9 @@ import io.airbyte.integrations.source.postgres.internal.models.XminStatus; import io.airbyte.integrations.source.postgres.xmin.XminCtidUtils.StreamsCategorised; import io.airbyte.integrations.source.relationaldb.state.StreamStateManager; -import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.Field; import io.airbyte.protocol.models.JsonSchemaType; +import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.v0.AirbyteStateMessage.AirbyteStateType; import io.airbyte.protocol.models.v0.AirbyteStreamState; import io.airbyte.protocol.models.v0.CatalogHelpers; @@ -31,20 +34,20 @@ public class XminCtidUtilsTest { private static final ConfiguredAirbyteStream MODELS_STREAM = CatalogHelpers.toDefaultConfiguredStream(CatalogHelpers.createAirbyteStream( - "MODELS_STREAM_NAME", - "MODELS_SCHEMA", - Field.of("COL_ID", JsonSchemaType.INTEGER), - Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), - Field.of("COL_MODEL", JsonSchemaType.STRING)) + "MODELS_STREAM_NAME", + "MODELS_SCHEMA", + Field.of("COL_ID", JsonSchemaType.INTEGER), + Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), + Field.of("COL_MODEL", JsonSchemaType.STRING)) .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) .withSourceDefinedPrimaryKey(List.of(List.of("COL_ID")))); private static final ConfiguredAirbyteStream MODELS_STREAM_2 = CatalogHelpers.toDefaultConfiguredStream(CatalogHelpers.createAirbyteStream( - "MODELS_STREAM_NAME_2", - "MODELS_SCHEMA", - Field.of("COL_ID", JsonSchemaType.INTEGER), - Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), - Field.of("COL_MODEL", JsonSchemaType.STRING)) + "MODELS_STREAM_NAME_2", + "MODELS_SCHEMA", + Field.of("COL_ID", JsonSchemaType.INTEGER), + Field.of("COL_MAKE_ID", JsonSchemaType.INTEGER), + Field.of("COL_MODEL", JsonSchemaType.STRING)) .withSupportedSyncModes(Lists.newArrayList(SyncMode.FULL_REFRESH, SyncMode.INCREMENTAL)) .withSourceDefinedPrimaryKey(List.of(List.of("COL_ID")))); From 97aa8ffb84024751b8188db5660e53e9edf95ba3 Mon Sep 17 00:00:00 2001 From: Subodh Kant Chaturvedi Date: Thu, 22 Jun 2023 13:25:38 +0530 Subject: [PATCH 27/27] source-postgres : Add logic to handle xmin wraparound (#27466) MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit * Initial logic for xmin wraparound * Add tests * Address comments * add xmin-wraparound check * address review comments * 🤖 Auto format source-postgres code [skip ci] * missed this * 🤖 Auto format source-postgres code [skip ci] --------- Co-authored-by: Akash Kulkarni Co-authored-by: Akash Kulkarni <113392464+akashkulk@users.noreply.github.com> Co-authored-by: octavia-squidington-iii --- .../connectors/source-postgres/build.gradle | 2 + .../source/postgres/PostgresSource.java | 2 +- .../postgres/xmin/PostgresXminHandler.java | 66 +++++++++++++++---- .../source/postgres/xmin/XminCtidUtils.java | 20 +++++- .../xmin/PostgresXminHandlerTest.java | 53 +++++++++++++++ .../postgres/xmin/XminCtidUtilsTest.java | 7 +- 6 files changed, 130 insertions(+), 20 deletions(-) create mode 100644 airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandlerTest.java diff --git a/airbyte-integrations/connectors/source-postgres/build.gradle b/airbyte-integrations/connectors/source-postgres/build.gradle index a28e207887e344..8219fcbf540afa 100644 --- a/airbyte-integrations/connectors/source-postgres/build.gradle +++ b/airbyte-integrations/connectors/source-postgres/build.gradle @@ -58,3 +58,5 @@ jsonSchema2Pojo { } + + 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 a809dc8fddce45..60a00ff6203b9d 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 @@ -459,7 +459,7 @@ public List> getIncrementalIterators(final AutoCloseableIterators.lazyIterator(incrementalIteratorSupplier, null))); } else if (PostgresUtils.isXmin(sourceConfig) && isIncrementalSyncMode(catalog)) { - final StreamsCategorised streamsCategorised = categoriseStreams(stateManager, catalog); + final StreamsCategorised streamsCategorised = categoriseStreams(stateManager, catalog, xminStatus); final List> ctidIterator = new ArrayList<>(); final List> xminIterator = new ArrayList<>(); diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandler.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandler.java index eac9cdd380b1d0..3ec0e3692ffd67 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandler.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandler.java @@ -7,6 +7,7 @@ import static io.airbyte.integrations.source.relationaldb.RelationalDbQueryUtils.getFullyQualifiedTableNameWithQuoting; import com.fasterxml.jackson.databind.JsonNode; +import com.google.common.annotations.VisibleForTesting; import io.airbyte.commons.stream.AirbyteStreamUtils; import io.airbyte.commons.util.AutoCloseableIterator; import io.airbyte.commons.util.AutoCloseableIterators; @@ -45,7 +46,7 @@ public class PostgresXminHandler { private final JdbcCompatibleSourceOperations sourceOperations; private final JdbcDatabase database; private final String quoteString; - private final XminStatus xminStatus; + private final XminStatus currentXminStatus; private final XminStateManager xminStateManager; private static final Logger LOGGER = LoggerFactory.getLogger(PostgresXminHandler.class); @@ -58,7 +59,7 @@ public PostgresXminHandler(final JdbcDatabase database, this.database = database; this.sourceOperations = sourceOperations; this.quoteString = quoteString; - this.xminStatus = xminStatus; + this.currentXminStatus = xminStatus; this.xminStateManager = xminStateManager; } @@ -138,27 +139,64 @@ private PreparedStatement createXminQueryStatement( quoteString); final String wrappedColumnNames = RelationalDbQueryUtils.enquoteIdentifierList(columnNames, quoteString); + + // Get the xmin status associated with the previous run + final XminStatus previousRunXminStatus = xminStateManager.getXminStatus(airbyteStream); + final PreparedStatement xminPreparedStatement = + getXminPreparedStatement(connection, wrappedColumnNames, fullTableName, previousRunXminStatus, currentXminStatus); + LOGGER.info("Executing query for table {}: {}", tableName, xminPreparedStatement); + return xminPreparedStatement; + } catch (final SQLException e) { + throw new RuntimeException(e); + } + } + + private PreparedStatement getXminPreparedStatement(final Connection connection, + final String wrappedColumnNames, + final String fullTableName, + final XminStatus prevRunXminStatus, + final XminStatus currentXminStatus) + throws SQLException { + + if (prevRunXminStatus == null) { + throw new RuntimeException("XminStatus not found for table " + fullTableName + ", should have triggered a full sync via ctid path"); + } else if (isSingleWraparound(prevRunXminStatus, currentXminStatus)) { + // The xmin state that we save represents the lowest XID that is still in progress. To make sure we + // don't miss data associated with the current transaction, we have to issue an >=. Because of the + // wraparound, the changes prior to the + // end xmin xid value must also be captured. + LOGGER.info("Detect a single wraparound for {}", fullTableName); + final String sql = String.format("SELECT %s FROM %s WHERE xmin::text::bigint >= ? OR xmin::text::bigint < ?", + wrappedColumnNames, fullTableName); + final PreparedStatement preparedStatement = connection.prepareStatement(sql); + preparedStatement.setLong(1, prevRunXminStatus.getXminXidValue()); + preparedStatement.setLong(2, currentXminStatus.getXminXidValue()); + + return preparedStatement; + } else { // The xmin state that we save represents the lowest XID that is still in progress. To make sure we - // don't miss - // data associated with the current transaction, we have to issue an >= + // don't miss data associated with the current transaction, we have to issue an >= final String sql = String.format("SELECT %s FROM %s WHERE xmin::text::bigint >= ?", wrappedColumnNames, fullTableName); final PreparedStatement preparedStatement = connection.prepareStatement(sql.toString()); + preparedStatement.setLong(1, prevRunXminStatus.getXminXidValue()); - final XminStatus currentStreamXminStatus = xminStateManager.getXminStatus(airbyteStream); - if (currentStreamXminStatus != null) { - preparedStatement.setLong(1, currentStreamXminStatus.getXminXidValue()); - } else { - preparedStatement.setLong(1, 0L); - } - LOGGER.info("Executing query for table {}: {}", tableName, preparedStatement); return preparedStatement; - } catch (final SQLException e) { - throw new RuntimeException(e); } } + @VisibleForTesting + static boolean isSingleWraparound(final XminStatus prevRunXminStatus, final XminStatus currentXminStatus) { + // Detect whether the source Postgres DB has undergone a single wraparound event. + return currentXminStatus.getNumWraparound() - prevRunXminStatus.getNumWraparound() == 1; + } + + static boolean shouldPerformFullSync(final XminStatus currentXminStatus, final JsonNode streamState) { + // Detects whether source Postgres DB has undergone multiple wraparound events between syncs. + return streamState.has("num_wraparound") && (currentXminStatus.getNumWraparound() - streamState.get("num_wraparound").asLong() >= 2); + } + // Transforms the given iterator to create an {@link AirbyteRecordMessage} private static AutoCloseableIterator getRecordIterator( final AutoCloseableIterator recordIterator, @@ -196,7 +234,7 @@ private AutoCloseableIterator augmentWithState(final AutoCloseab autoCloseableIterator -> new XminStateIterator( autoCloseableIterator, pair, - xminStatus), + currentXminStatus), recordIterator, AirbyteStreamUtils.convertFromNameAndNamespace(pair.getName(), pair.getNamespace())); } diff --git a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java index 6ff588fa2dfbf2..1eef23b6616d9a 100644 --- a/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java +++ b/airbyte-integrations/connectors/source-postgres/src/main/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtils.java @@ -4,9 +4,12 @@ package io.airbyte.integrations.source.postgres.xmin; +import static io.airbyte.integrations.source.postgres.xmin.PostgresXminHandler.shouldPerformFullSync; + import com.fasterxml.jackson.databind.JsonNode; import com.google.common.collect.Sets; import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.postgres.internal.models.XminStatus; import io.airbyte.integrations.source.relationaldb.state.StateManager; import io.airbyte.protocol.models.v0.AirbyteStateMessage; import io.airbyte.protocol.models.v0.AirbyteStreamNameNamespacePair; @@ -18,6 +21,8 @@ import java.util.List; import java.util.Set; import java.util.stream.Collectors; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * The class mainly categorises the streams based on the state type into two categories : 1. Streams @@ -27,7 +32,11 @@ */ public class XminCtidUtils { - public static StreamsCategorised categoriseStreams(final StateManager stateManager, final ConfiguredAirbyteCatalog fullCatalog) { + private static final Logger LOGGER = LoggerFactory.getLogger(XminCtidUtils.class); + + public static StreamsCategorised categoriseStreams(final StateManager stateManager, + final ConfiguredAirbyteCatalog fullCatalog, + final XminStatus currentXminStatus) { final List rawStateMessages = stateManager.getRawStateMessages(); final List statesFromCtidSync = new ArrayList<>(); final List statesFromXminSync = new ArrayList<>(); @@ -48,7 +57,14 @@ public static StreamsCategorised categoriseStreams(final StateManager stateManag statesFromCtidSync.add(stateMessage); streamsStillInCtidSync.add(new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), streamDescriptor.getNamespace())); } else if (streamState.get("state_type").asText().equalsIgnoreCase("xmin")) { - statesFromXminSync.add(stateMessage); + if (shouldPerformFullSync(currentXminStatus, streamState)) { + final AirbyteStreamNameNamespacePair pair = new AirbyteStreamNameNamespacePair(streamDescriptor.getName(), + streamDescriptor.getNamespace()); + LOGGER.info("Detected multiple wraparounds. Will perform a full sync for {}", pair); + streamsStillInCtidSync.add(pair); + } else { + statesFromXminSync.add(stateMessage); + } } else { throw new RuntimeException("Unknown state type: " + streamState.get("state_type").asText()); } diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandlerTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandlerTest.java new file mode 100644 index 00000000000000..25c2aad834a8d3 --- /dev/null +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/PostgresXminHandlerTest.java @@ -0,0 +1,53 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.source.postgres.xmin; + +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +import com.fasterxml.jackson.databind.JsonNode; +import io.airbyte.commons.json.Jsons; +import io.airbyte.integrations.source.postgres.internal.models.XminStatus; +import org.junit.jupiter.api.Test; + +public class PostgresXminHandlerTest { + + @Test + void testWraparound() { + final XminStatus initialStatus = + new XminStatus() + .withNumWraparound(0L) + .withXminRawValue(5555L) + .withXminRawValue(5555L); + final JsonNode initialStatusAsJson = Jsons.jsonNode(initialStatus); + + final XminStatus noWrapAroundStatus = + new XminStatus() + .withNumWraparound(0L) + .withXminRawValue(5588L) + .withXminRawValue(5588L); + assertFalse(PostgresXminHandler.isSingleWraparound(initialStatus, noWrapAroundStatus)); + assertFalse(PostgresXminHandler.shouldPerformFullSync(noWrapAroundStatus, initialStatusAsJson)); + + final XminStatus singleWrapAroundStatus = + new XminStatus() + .withNumWraparound(1L) + .withXminRawValue(5588L) + .withXminRawValue(4294972884L); + + assertTrue(PostgresXminHandler.isSingleWraparound(initialStatus, singleWrapAroundStatus)); + assertFalse(PostgresXminHandler.shouldPerformFullSync(singleWrapAroundStatus, initialStatusAsJson)); + + final XminStatus doubleWrapAroundStatus = + new XminStatus() + .withNumWraparound(2L) + .withXminRawValue(5588L) + .withXminRawValue(8589940180L); + + assertFalse(PostgresXminHandler.isSingleWraparound(initialStatus, doubleWrapAroundStatus)); + assertTrue(PostgresXminHandler.shouldPerformFullSync(doubleWrapAroundStatus, initialStatusAsJson)); + } + +} diff --git a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java index 6e34678023e31b..6a5ceb13cb1d77 100644 --- a/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java +++ b/airbyte-integrations/connectors/source-postgres/src/test/java/io/airbyte/integrations/source/postgres/xmin/XminCtidUtilsTest.java @@ -54,9 +54,10 @@ public class XminCtidUtilsTest { @Test public void emptyStateTest() { final ConfiguredAirbyteCatalog configuredCatalog = new ConfiguredAirbyteCatalog().withStreams(Arrays.asList(MODELS_STREAM, MODELS_STREAM_2)); - + final XminStatus xminStatus = new XminStatus().withStateType(StateType.XMIN).withVersion(2L).withXminXidValue(9L).withXminRawValue(9L) + .withNumWraparound(1L); final StreamStateManager streamStateManager = new StreamStateManager(Collections.emptyList(), configuredCatalog); - final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog); + final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog, xminStatus); assertTrue(streamsCategorised.xminStreams().streamsForXminSync().isEmpty()); assertTrue(streamsCategorised.xminStreams().statesFromXminSync().isEmpty()); @@ -82,7 +83,7 @@ public void correctCategorisationTest() { new StreamDescriptor().withName(MODELS_STREAM_2.getStream().getName()).withNamespace(MODELS_STREAM_2.getStream().getNamespace())); final StreamStateManager streamStateManager = new StreamStateManager(Arrays.asList(xminState, ctidState), configuredCatalog); - final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog); + final StreamsCategorised streamsCategorised = XminCtidUtils.categoriseStreams(streamStateManager, configuredCatalog, xminStatus); assertEquals(1, streamsCategorised.xminStreams().streamsForXminSync().size()); assertEquals(MODELS_STREAM, streamsCategorised.xminStreams().streamsForXminSync().get(0));