From afa480514a7771339902287e5ffb7c0a5aa14947 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Fri, 21 Jun 2024 11:43:05 -0700 Subject: [PATCH] add generation_id --- .../destination-redshift/build.gradle | 2 +- .../destination-redshift/metadata.yaml | 2 +- .../redshift/RedshiftDestination.kt | 36 ++---- .../RedshiftStagingStorageOperation.kt | 6 +- .../typing_deduping/RedshiftDV2Migration.kt | 1 + .../RedshiftDestinationHandler.kt | 23 ++-- .../RedshiftGenerationIdMigration.kt | 109 ++++++++++++++++ .../typing_deduping/RedshiftSqlGenerator.kt | 15 ++- .../redshift/typing_deduping/RedshiftState.kt | 7 +- .../AbstractRedshiftTypingDedupingTest.kt | 118 ++++++++++++++++++ ...orchange_expectedrecords_dedup_final.jsonl | 6 +- ...rsorchange_expectedrecords_dedup_raw.jsonl | 8 +- .../sync1_expectedrecords_dedup_final.jsonl | 8 +- .../sync1_expectedrecords_dedup_final2.jsonl | 2 +- ...sync1_expectedrecords_nondedup_final.jsonl | 10 +- .../dat/sync1_expectedrecords_raw.jsonl | 10 +- .../dat/sync1_expectedrecords_raw2.jsonl | 2 +- ...nc1_recordnull_expectedrecords_final.jsonl | 4 +- ...sync1_recordnull_expectedrecords_raw.jsonl | 4 +- ...ectedrecords_incremental_dedup_final.jsonl | 4 +- ...xpectedrecords_incremental_dedup_raw.jsonl | 14 +-- ...ctedrecords_fullrefresh_append_final.jsonl | 16 +-- ..._fullrefresh_append_mixed_meta_final.jsonl | 8 +- ...drecords_fullrefresh_overwrite_final.jsonl | 6 +- ...tedrecords_fullrefresh_overwrite_raw.jsonl | 6 +- ...ectedrecords_incremental_dedup_final.jsonl | 6 +- ...ctedrecords_incremental_dedup_final2.jsonl | 2 +- ...records_incremental_dedup_meta_final.jsonl | 4 +- ...sync2_expectedrecords_mixed_meta_raw.jsonl | 8 +- .../sync2_expectedrecords_overwrite_raw.jsonl | 4 + .../dat/sync2_expectedrecords_raw.jsonl | 16 +-- .../dat/sync2_expectedrecords_raw2.jsonl | 4 +- .../alltypes_expectedrecords_final.jsonl | 10 +- .../alltypes_expectedrecords_raw.jsonl | 10 +- .../alltypes_v1v2_expectedrecords_final.jsonl | 10 +- .../alltypes_v1v2_expectedrecords_raw.jsonl | 10 +- ...crementaldedup_expectedrecords_final.jsonl | 4 +- ...ypes_in_string_expectedrecords_final.jsonl | 10 +- ...casecolumnname_expectedrecords_final.jsonl | 2 +- .../nocolumns_expectedrecords_final.jsonl | 2 +- ...servedkeywords_expectedrecords_final.jsonl | 2 +- ...mestampformats_expectedrecords_final.jsonl | 20 +-- ...irdcolumnnames_expectedrecords_final.jsonl | 2 +- .../resources/typing_deduping_with_cdc.sql | 7 +- docs/integrations/destinations/redshift.md | 1 + 45 files changed, 398 insertions(+), 163 deletions(-) create mode 100644 airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftGenerationIdMigration.kt create mode 100644 airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_overwrite_raw.jsonl diff --git a/airbyte-integrations/connectors/destination-redshift/build.gradle b/airbyte-integrations/connectors/destination-redshift/build.gradle index 975ce629bfcd..2da285d20c34 100644 --- a/airbyte-integrations/connectors/destination-redshift/build.gradle +++ b/airbyte-integrations/connectors/destination-redshift/build.gradle @@ -4,7 +4,7 @@ plugins { } airbyteJavaConnector { - cdkVersionRequired = '0.35.16' + cdkVersionRequired = '0.38.3' features = ['db-destinations', 's3-destinations', 'typing-deduping'] useLocalCdk = false } diff --git a/airbyte-integrations/connectors/destination-redshift/metadata.yaml b/airbyte-integrations/connectors/destination-redshift/metadata.yaml index 67233c3c4f0f..dd69ea37cd1a 100644 --- a/airbyte-integrations/connectors/destination-redshift/metadata.yaml +++ b/airbyte-integrations/connectors/destination-redshift/metadata.yaml @@ -5,7 +5,7 @@ data: connectorSubtype: database connectorType: destination definitionId: f7a7d195-377f-cf5b-70a5-be6b819019dc - dockerImageTag: 3.1.1 + dockerImageTag: 3.2.0 dockerRepository: airbyte/destination-redshift documentationUrl: https://docs.airbyte.com/integrations/destinations/redshift githubIssueLabel: destination-redshift diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/RedshiftDestination.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/RedshiftDestination.kt index 52f803456194..3792ae154601 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/RedshiftDestination.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/RedshiftDestination.kt @@ -35,7 +35,6 @@ import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordM import io.airbyte.cdk.integrations.destination.async.state.FlushFailure import io.airbyte.cdk.integrations.destination.jdbc.AbstractJdbcDestination.Companion.DISABLE_TYPE_DEDUPE import io.airbyte.cdk.integrations.destination.jdbc.AbstractJdbcDestination.Companion.RAW_SCHEMA_OVERRIDE -import io.airbyte.cdk.integrations.destination.jdbc.typing_deduping.JdbcDestinationHandler import io.airbyte.cdk.integrations.destination.s3.AesCbcEnvelopeEncryption import io.airbyte.cdk.integrations.destination.s3.EncryptionConfig import io.airbyte.cdk.integrations.destination.s3.EncryptionConfig.Companion.fromJson @@ -61,6 +60,7 @@ import io.airbyte.integrations.destination.redshift.constants.RedshiftDestinatio import io.airbyte.integrations.destination.redshift.operation.RedshiftStagingStorageOperation import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftDV2Migration import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftDestinationHandler +import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftGenerationIdMigration import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftRawTableAirbyteMetaMigration import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftSqlGenerator import io.airbyte.integrations.destination.redshift.typing_deduping.RedshiftState @@ -69,6 +69,7 @@ import io.airbyte.integrations.destination.redshift.util.RedshiftUtil import io.airbyte.protocol.models.v0.AirbyteConnectionStatus import io.airbyte.protocol.models.v0.AirbyteMessage import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta +import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage.AirbyteStreamStatus import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.airbyte.protocol.models.v0.ConnectorSpecification import io.airbyte.protocol.models.v0.DestinationSyncMode @@ -81,7 +82,6 @@ import java.util.concurrent.Executors import java.util.function.Consumer import javax.sql.DataSource import org.apache.commons.lang3.NotImplementedException -import org.apache.commons.lang3.StringUtils import org.jetbrains.annotations.VisibleForTesting import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -187,7 +187,11 @@ class RedshiftDestination : BaseConnector(), Destination { isSchemaMismatch = true, isFinalTableEmpty = true, destinationState = - RedshiftState(needsSoftReset = false, isAirbyteMetaPresentInRaw = true), + RedshiftState( + needsSoftReset = false, + isAirbyteMetaPresentInRaw = true, + isGenerationIdPresent = true, + ), ), FileUploadFormat.CSV, destinationColumns, @@ -211,7 +215,7 @@ class RedshiftDestination : BaseConnector(), Destination { ) streamOperation.finalizeTable( streamConfig, - StreamSyncSummary(recordsWritten = Optional.of(1)), + StreamSyncSummary(recordsWritten = 1, AirbyteStreamStatus.COMPLETE), ) // And now that we have a table, simulate the next sync startup. @@ -315,14 +319,6 @@ class RedshiftDestination : BaseConnector(), Destination { return RedshiftSqlGenerator(namingResolver, config) } - private fun getDestinationHandler( - databaseName: String, - database: JdbcDatabase, - rawTableSchema: String - ): JdbcDestinationHandler { - return RedshiftDestinationHandler(databaseName, database, rawTableSchema) - } - private fun getMigrations( database: JdbcDatabase, databaseName: String, @@ -336,6 +332,7 @@ class RedshiftDestination : BaseConnector(), Destination { sqlGenerator, ), RedshiftRawTableAirbyteMetaMigration(database, databaseName), + RedshiftGenerationIdMigration(database, databaseName) ) } @@ -372,13 +369,7 @@ class RedshiftDestination : BaseConnector(), Destination { else NoEncryption() val s3Options = RedshiftUtil.findS3Options(config) val s3Config: S3DestinationConfig = S3DestinationConfig.getS3DestinationConfig(s3Options) - val defaultNamespace = config["schema"].asText() - for (stream in catalog.streams) { - if (StringUtils.isEmpty(stream.stream.namespace)) { - stream.stream.namespace = defaultNamespace - } - } val sqlGenerator = RedshiftSqlGenerator(namingResolver, config) val parsedCatalog: ParsedCatalog @@ -388,10 +379,10 @@ class RedshiftDestination : BaseConnector(), Destination { val rawNamespace: String if (getRawNamespaceOverride(RAW_SCHEMA_OVERRIDE).isPresent) { rawNamespace = getRawNamespaceOverride(RAW_SCHEMA_OVERRIDE).get() - catalogParser = CatalogParser(sqlGenerator, rawNamespace) + catalogParser = CatalogParser(sqlGenerator, defaultNamespace, rawNamespace) } else { rawNamespace = JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE - catalogParser = CatalogParser(sqlGenerator, rawNamespace) + catalogParser = CatalogParser(sqlGenerator, defaultNamespace, rawNamespace) } val redshiftDestinationHandler = RedshiftDestinationHandler(databaseName, database, rawNamespace) @@ -436,8 +427,7 @@ class RedshiftDestination : BaseConnector(), Destination { }, onFlush = DefaultFlush(OPTIMAL_FLUSH_BATCH_SIZE, syncOperation), catalog, - BufferManager(bufferMemoryLimit), - Optional.ofNullable(defaultNamespace), + BufferManager(defaultNamespace, bufferMemoryLimit), FlushFailure(), Executors.newFixedThreadPool(5), AirbyteMessageDeserializer(getDataTransformer(parsedCatalog, defaultNamespace)), @@ -463,7 +453,7 @@ class RedshiftDestination : BaseConnector(), Destination { "com.amazon.redshift.ssl.NonValidatingFactory" ) - private val destinationColumns = JavaBaseConstants.DestinationColumns.V2_WITH_META + private val destinationColumns = JavaBaseConstants.DestinationColumns.V2_WITH_GENERATION private const val OPTIMAL_FLUSH_BATCH_SIZE: Long = 50 * 1024 * 1024 private val bufferMemoryLimit: Long = (Runtime.getRuntime().maxMemory() * 0.5).toLong() diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/operation/RedshiftStagingStorageOperation.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/operation/RedshiftStagingStorageOperation.kt index fd1d80f78ff9..611fa189dea5 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/operation/RedshiftStagingStorageOperation.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/operation/RedshiftStagingStorageOperation.kt @@ -51,7 +51,8 @@ class RedshiftStagingStorageOperation( s3StorageOperations.createBucketIfNotExists() } - override fun writeToStage(streamId: StreamId, data: SerializableBuffer) { + override fun writeToStage(streamConfig: StreamConfig, data: SerializableBuffer) { + val streamId = streamConfig.id val objectPath: String = getStagingPath(streamId) log.info { "Uploading records to for ${streamId.rawNamespace}.${streamId.rawName} to path $objectPath" @@ -201,7 +202,8 @@ class RedshiftStagingStorageOperation( ${JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT} TIMESTAMPTZ DEFAULT GETDATE(), ${JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT} TIMESTAMPTZ, ${JavaBaseConstants.COLUMN_NAME_DATA} SUPER NOT NULL, - ${JavaBaseConstants.COLUMN_NAME_AB_META} SUPER NULL + ${JavaBaseConstants.COLUMN_NAME_AB_META} SUPER NULL, + ${JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID} BIGINT NULL ) """.trimIndent() } diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDV2Migration.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDV2Migration.kt index 1c1c48806b11..f2d315080315 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDV2Migration.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDV2Migration.kt @@ -34,6 +34,7 @@ class RedshiftDV2Migration( RedshiftState( needsSoftReset = false, isAirbyteMetaPresentInRaw = false, + isGenerationIdPresent = false ), true, ) diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDestinationHandler.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDestinationHandler.kt index 9c584b20599e..32194aa06321 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftDestinationHandler.kt @@ -90,12 +90,9 @@ class RedshiftDestinationHandler( for (transaction in transactions) { val transactionId = UUID.randomUUID() if (logStatements) { - log.info( - "Executing sql {}-{}: {}", - queryId, - transactionId, - java.lang.String.join("\n", transaction) - ) + log.info { + "Executing sql $queryId-$transactionId: ${transaction.joinToString("\n")}" + } } val startTime = System.currentTimeMillis() @@ -113,7 +110,7 @@ class RedshiftDestinationHandler( logStatements = logStatements ) } catch (e: SQLException) { - log.error("Sql {}-{} failed", queryId, transactionId, e) + log.error(e) { "Sql $queryId-$transactionId failed" } // This is a big hammer for something that should be much more targetted, only when // executing the // DROP TABLE command. @@ -129,12 +126,9 @@ class RedshiftDestinationHandler( throw e } - log.info( - "Sql {}-{} completed in {} ms", - queryId, - transactionId, - System.currentTimeMillis() - startTime - ) + log.info { + "Sql $queryId-$transactionId completed in ${System.currentTimeMillis() - startTime} ms" + } } } @@ -156,7 +150,8 @@ class RedshiftDestinationHandler( return RedshiftState( json.hasNonNull("needsSoftReset") && json["needsSoftReset"].asBoolean(), json.hasNonNull("isAirbyteMetaPresentInRaw") && - json["isAirbyteMetaPresentInRaw"].asBoolean() + json["isAirbyteMetaPresentInRaw"].asBoolean(), + json.hasNonNull("isGenerationIdPresent") && json["isGenerationIdPresent"].asBoolean(), ) } diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftGenerationIdMigration.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftGenerationIdMigration.kt new file mode 100644 index 000000000000..3566b36f2130 --- /dev/null +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftGenerationIdMigration.kt @@ -0,0 +1,109 @@ +/* + * Copyright (c) 2024 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.redshift.typing_deduping + +import com.fasterxml.jackson.databind.JsonNode +import io.airbyte.cdk.db.jdbc.JdbcDatabase +import io.airbyte.cdk.integrations.base.JavaBaseConstants +import io.airbyte.integrations.base.destination.typing_deduping.DestinationHandler +import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus +import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig +import io.airbyte.integrations.base.destination.typing_deduping.migrators.Migration +import io.github.oshai.kotlinlogging.KotlinLogging + +private val logger = KotlinLogging.logger {} + +class RedshiftGenerationIdMigration( + private val database: JdbcDatabase, + private val databaseName: String, +) : Migration { + override fun migrateIfNecessary( + destinationHandler: DestinationHandler, + stream: StreamConfig, + state: DestinationInitialStatus + ): Migration.MigrationResult { + if (state.destinationState.isGenerationIdPresent) { + logger.info { + "Skipping generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because our state says it's already done" + } + return Migration.MigrationResult(state.destinationState, invalidateInitialState = false) + } + + if (!state.initialRawTableStatus.rawTableExists) { + // The raw table doesn't exist. No migration necessary. Update the state. + logger.info { + "Skipping generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because the raw table doesn't exist" + } + return Migration.MigrationResult( + state.destinationState.copy(isGenerationIdPresent = true), + invalidateInitialState = false + ) + } + + // Add generation_id to the raw table if necessary + val rawTableDefinitionQueryResult: List = + database.queryJsons( + """ + SHOW COLUMNS + FROM TABLE "$databaseName"."${stream.id.rawNamespace}"."${stream.id.rawName}" + LIKE '${JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID}' + """.trimIndent() + ) + if (rawTableDefinitionQueryResult.isNotEmpty()) { + logger.info { + "${stream.id.originalNamespace}.${stream.id.originalName}: Skipping generation_id migration for raw table because it already has the generation_id column" + } + } else { + logger.info { + "Migrating generation_id for table ${stream.id.rawNamespace}.${stream.id.rawName}" + } + // Quote for raw table columns + val alterRawTableSql = + """ + ALTER TABLE "${stream.id.rawNamespace}"."${stream.id.rawName}" + ADD COLUMN "${JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID}" BIGINT; + """.trimIndent() + database.execute(alterRawTableSql) + } + + // Add generation_id to the final table if necessary + // As a slight optimization, only do this if we previously detected that the final table + // schema is wrong + if (state.isFinalTablePresent && state.isSchemaMismatch) { + val finalTableColumnQueryResult: List = + database.queryJsons( + """ + SHOW COLUMNS + FROM TABLE "$databaseName"."${stream.id.finalNamespace}"."${stream.id.finalName}" + LIKE '${JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID}' + """.trimIndent() + ) + if (finalTableColumnQueryResult.isNotEmpty()) { + logger.info { + "${stream.id.originalNamespace}.${stream.id.originalName}: Skipping generation_id migration for final table because it already has the generation_id column" + } + } else { + logger.info { + "Migrating generation_id for table ${stream.id.finalNamespace}.${stream.id.finalName}" + } + database.execute( + """ + ALTER TABLE "${stream.id.finalNamespace}"."${stream.id.finalName}" + ADD COLUMN "${JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID}" BIGINT NULL; + """.trimIndent() + ) + } + } else { + logger.info { + "${stream.id.originalNamespace}.${stream.id.originalName}: Skipping generation_id migration for final table. Final table exists: ${state.isFinalTablePresent}; final table schema is incorrect: ${state.isSchemaMismatch}" + } + } + + return Migration.MigrationResult( + state.destinationState.copy(isGenerationIdPresent = true), + invalidateInitialState = true + ) + } +} diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftSqlGenerator.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftSqlGenerator.kt index d4b77e2174e2..811ad7386c07 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftSqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftSqlGenerator.kt @@ -17,7 +17,7 @@ import io.airbyte.integrations.base.destination.typing_deduping.UnsupportedOneOf import io.airbyte.integrations.destination.redshift.constants.RedshiftDestinationConstants import io.airbyte.protocol.models.AirbyteRecordMessageMetaChange import java.sql.Timestamp -import java.util.* +import java.util.Optional import java.util.stream.Collectors import org.jooq.Condition import org.jooq.DataType @@ -146,7 +146,7 @@ open class RedshiftSqlGenerator( * @param arrays * @return */ - fun arrayConcatStmt(arrays: List?>): Field<*>? { + private fun arrayConcatStmt(arrays: List?>): Field<*>? { if (arrays.isEmpty()) { return DSL.field("ARRAY()") // Return an empty string if the list is empty } @@ -165,7 +165,7 @@ open class RedshiftSqlGenerator( return result } - fun toCastingErrorCaseStmt(column: ColumnId, type: AirbyteType): Field<*> { + private fun toCastingErrorCaseStmt(column: ColumnId, type: AirbyteType): Field<*> { val field: Field<*> = DSL.field(DSL.quotedName(JavaBaseConstants.COLUMN_NAME_DATA, column.originalName)) // Just checks if data is not null but casted data is null. This also accounts for @@ -260,7 +260,14 @@ open class RedshiftSqlGenerator( "OBJECT", superType, DSL.`val`(AIRBYTE_META_COLUMN_CHANGES_KEY), - airbyteMetaChangesArray + airbyteMetaChangesArray, + DSL.`val`(JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY), + DSL.field( + DSL.quotedName( + JavaBaseConstants.COLUMN_NAME_AB_META, + JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY + ) + ), ) .`as`(JavaBaseConstants.COLUMN_NAME_AB_META) } diff --git a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftState.kt b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftState.kt index 056b3323046e..0b0b845c16ea 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftState.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/main/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/RedshiftState.kt @@ -6,8 +6,11 @@ package io.airbyte.integrations.destination.redshift.typing_deduping import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState -data class RedshiftState(val needsSoftReset: Boolean, val isAirbyteMetaPresentInRaw: Boolean) : - MinimumDestinationState { +data class RedshiftState( + val needsSoftReset: Boolean, + val isAirbyteMetaPresentInRaw: Boolean, + val isGenerationIdPresent: Boolean, +) : MinimumDestinationState { override fun needsSoftReset(): Boolean { return needsSoftReset } diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/AbstractRedshiftTypingDedupingTest.kt b/airbyte-integrations/connectors/destination-redshift/src/test-integration/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/AbstractRedshiftTypingDedupingTest.kt index e9d8c18e1c63..6f745e418f79 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/AbstractRedshiftTypingDedupingTest.kt +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/kotlin/io/airbyte/integrations/destination/redshift/typing_deduping/AbstractRedshiftTypingDedupingTest.kt @@ -7,8 +7,11 @@ import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode import edu.umd.cs.findbugs.annotations.SuppressFBWarnings import io.airbyte.cdk.db.JdbcCompatibleSourceOperations +import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.standardtest.destination.typing_deduping.JdbcTypingDedupingTest import io.airbyte.commons.json.Jsons.deserialize +import io.airbyte.integrations.base.destination.typing_deduping.BaseTypingDedupingTest +import io.airbyte.integrations.base.destination.typing_deduping.BaseTypingDedupingTest.Companion import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator import io.airbyte.integrations.destination.redshift.RedshiftDestination import io.airbyte.integrations.destination.redshift.RedshiftSQLNameTransformer @@ -24,6 +27,7 @@ import javax.sql.DataSource import org.jooq.DSLContext import org.jooq.conf.Settings import org.jooq.impl.DSL +import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Disabled import org.junit.jupiter.api.Test @@ -170,6 +174,9 @@ abstract class AbstractRedshiftTypingDedupingTest : JdbcTypingDedupingTest() { .withStreams( List.of( ConfiguredAirbyteStream() + .withSyncId(42) + .withGenerationId(43) + .withMinimumGenerationId(0) .withSyncMode(SyncMode.FULL_REFRESH) .withDestinationSyncMode(DestinationSyncMode.OVERWRITE) .withStream( @@ -201,6 +208,117 @@ abstract class AbstractRedshiftTypingDedupingTest : JdbcTypingDedupingTest() { verifySyncResult(expectedRawRecords, expectedFinalRecords, disableFinalTableComparison()) } + @Test + @Throws(Exception::class) + fun testGenerationIdMigrationForAppend() { + val catalog = + ConfiguredAirbyteCatalog() + .withStreams( + listOf( + ConfiguredAirbyteStream() + .withSyncMode(SyncMode.FULL_REFRESH) + .withDestinationSyncMode(DestinationSyncMode.APPEND) + .withSyncId(42L) + .withGenerationId(43L) + .withMinimumGenerationId(0L) + .withStream( + AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(SCHEMA) + ) + ) + ) + + // First sync + val messages1 = readMessages("dat/sync1_messages.jsonl") + runSync( + catalog, + messages1, + "airbyte/destination-redshift:3.1.1", + ) + + // Second sync + val messages2 = readMessages("dat/sync2_messages.jsonl") + runSync(catalog, messages2) + + // The first 5 records in these files were written by the old version, + // which does not write _airbyte_generation_id to the raw/final tables, + // and does not write sync_id to _airbyte_meta. + // So modify the expected records to reflect those differences. + val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_raw.jsonl") + for (i in 0..4) { + val record = expectedRawRecords2[i] as ObjectNode + record.remove(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID) + (record.get(JavaBaseConstants.COLUMN_NAME_AB_META) as ObjectNode).remove( + JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY + ) + } + val expectedFinalRecords2 = + readRecords("dat/sync2_expectedrecords_fullrefresh_append_final.jsonl") + for (i in 0..4) { + val record = expectedFinalRecords2[i] as ObjectNode + record.remove(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID) + (record.get(JavaBaseConstants.COLUMN_NAME_AB_META) as ObjectNode).remove( + JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY + ) + } + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison()) + + // Verify that we didn't trigger a soft reset. + // There should be two unique loaded_at values in the raw table. + // (only do this if T+D is enabled to begin with; otherwise loaded_at will just be null) + if (!disableFinalTableComparison()) { + val actualRawRecords2 = dumpRawTableRecords(streamNamespace, streamName) + val loadedAtValues = + actualRawRecords2 + .map { record: JsonNode -> + record.get(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT) + } + .toSet() + Assertions.assertEquals( + 2, + loadedAtValues.size, + "Expected two different values for loaded_at. If there is only 1 value, then we incorrectly triggered a soft reset. If there are more than 2, then something weird happened?" + ) + } + } + + @Test + fun testGenerationIdMigrationForOverwrite() { + val catalog = + ConfiguredAirbyteCatalog() + .withStreams( + listOf( + ConfiguredAirbyteStream() + .withSyncMode(SyncMode.FULL_REFRESH) + .withDestinationSyncMode(DestinationSyncMode.OVERWRITE) + .withSyncId(42L) + .withGenerationId(43L) + .withMinimumGenerationId(0L) + .withStream( + AirbyteStream() + .withNamespace(streamNamespace) + .withName(streamName) + .withJsonSchema(BaseTypingDedupingTest.Companion.SCHEMA), + ), + ), + ) + + // First sync + val messages1 = readMessages("dat/sync1_messages.jsonl") + runSync(catalog, messages1, "airbyte/destination-redshift:3.1.1") + + // Second sync + val messages2 = readMessages("dat/sync2_messages.jsonl") + runSync(catalog, messages2) + + val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_overwrite_raw.jsonl") + val expectedFinalRecords2 = + readRecords("dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl") + verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison()) + } + protected fun generateRandomString(totalLength: Int): String { return RANDOM.ints('a'.code, 'z'.code + 1) .limit(totalLength.toLong()) diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_final.jsonl index 89913b828215..f62fbeb08acf 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_final.jsonl @@ -1,3 +1,3 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "old_cursor": 1, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "old_cursor": 1, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_raw.jsonl index 9eb65ecf771a..2f97b055c2ba 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_cursorchange_expectedrecords_dedup_raw.jsonl @@ -1,4 +1,4 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final.jsonl index 387f7776cebf..7ba6be8e4bb7 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final.jsonl @@ -1,5 +1,5 @@ // Keep the Alice record with more recent updated_at -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final2.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final2.jsonl index 1b29b504aadd..7b5f7a1facef 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final2.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_dedup_final2.jsonl @@ -1 +1 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00.000000Z", "name": "Someone completely different"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00.000000Z", "name": "Someone completely different"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_nondedup_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_nondedup_final.jsonl index 8a9002dad5f6..c2c780a9d49b 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_nondedup_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_nondedup_final.jsonl @@ -1,6 +1,6 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} // Invalid columns are nulled out (i.e. SQL null, not JSON null) -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw.jsonl index 442ba0192225..d6a32df916aa 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw.jsonl @@ -1,6 +1,6 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} // Invalid data is still allowed in the raw table. -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"changes": [{"field": "address", "change": "NULLED", "reason": "SOURCE_RETRIEVAL_ERROR"}]}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": [{"field": "address", "change": "NULLED", "reason": "SOURCE_RETRIEVAL_ERROR"}]}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw2.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw2.jsonl index abbb44d6df67..6a13529ec2e3 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw2.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_expectedrecords_raw2.jsonl @@ -1 +1 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_final.jsonl index 4b2592985d1f..98a036b92574 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_final.jsonl @@ -1,3 +1,3 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "PLACE_HOLDER", "address": {"city": "San Francisco", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "PLACE_HOLDER", "address": {"city": "San Francisco", "state": "CA"}} // name is SQL null after nulling the record before persisting it. -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[{"field":"$.name","change":"NULLED","reason":"DESTINATION_FIELD_SIZE_LIMITATION"}]}, "id1": 2, "id2": 201, "updated_at": "2000-01-01T00:00:00.000000Z", "address": {"city": "New York", "state": "NY"}} \ No newline at end of file +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"$.name","change":"NULLED","reason":"DESTINATION_FIELD_SIZE_LIMITATION"}]}, "id1": 2, "id2": 201, "updated_at": "2000-01-01T00:00:00.000000Z", "address": {"city": "New York", "state": "NY"}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_raw.jsonl index b6746d3f906e..20178d571416 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync1_recordnull_expectedrecords_raw.jsonl @@ -1,2 +1,2 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "PLACE_HOLDER", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 201, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": null, "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta":{"changes":[{"field":"$.name","change":"NULLED","reason":"DESTINATION_FIELD_SIZE_LIMITATION"}]}} \ No newline at end of file +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "PLACE_HOLDER", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 201, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": null, "address": {"city": "New York", "state": "NY"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"$.name","change":"NULLED","reason":"DESTINATION_FIELD_SIZE_LIMITATION"}]}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_final.jsonl index 31af3c2967a8..6520a1c79fda 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_final.jsonl @@ -1,3 +1,3 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} // Charlie wasn't re-emitted with updated_at, so it still has a null cursor -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 2, "id2": 200, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 2, "id2": 200, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_raw.jsonl index dbf989c5b83a..7dc1f622d446 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_cursorchange_expectedrecords_incremental_dedup_raw.jsonl @@ -1,7 +1,7 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_final.jsonl index 0d83b1e66369..3f733de63c0b 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_final.jsonl @@ -1,9 +1,9 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00.000000Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00.000000Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00.000000Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_mixed_meta_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_mixed_meta_final.jsonl index b040bc169783..907cb3c5983d 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_mixed_meta_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_append_mixed_meta_final.jsonl @@ -4,7 +4,7 @@ {"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"errors":[]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} {"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"errors":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23", "address": {"city": "San Francisco", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23", "address": {"city": "San Francisco", "state": "CA"}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl index 0a4deced5cef..9c14b92152b4 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl @@ -1,3 +1,3 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00.000000Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00.000000Z"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl index fbf2611fe68e..bcfa3c5e8c56 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_fullrefresh_overwrite_raw.jsonl @@ -1,3 +1,3 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final.jsonl index c259b5206fb9..65bb7a58fd88 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final.jsonl @@ -1,4 +1,4 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} // Delete Bob, keep Charlie -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final2.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final2.jsonl index 69eeec6bab90..a2ea1317e982 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final2.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_final2.jsonl @@ -1 +1 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00.000000Z", "name": "Someone completely different v2"} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00.000000Z", "name": "Someone completely different v2"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_meta_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_meta_final.jsonl index 3f1e127f3646..92e93d490750 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_meta_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_incremental_dedup_meta_final.jsonl @@ -1,5 +1,5 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[]}, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00.000000Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}} // Delete Bob, updated Charlie -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_meta":{"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23", "address": {"city": "San Francisco", "state": "CA"}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}, "id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00.000000Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23", "address": {"city": "San Francisco", "state": "CA"}} // Record before meta in raw table will continue to have errors. {"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_meta": {"errors":[]}, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00.000000Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_mixed_meta_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_mixed_meta_raw.jsonl index 0203c7194f40..0dc68689f0af 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_mixed_meta_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_mixed_meta_raw.jsonl @@ -5,7 +5,7 @@ {"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}} {"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}} // And append the records from the second sync -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00Z", "name":"Charlie", "age":42, "registration_date":"2023-12-23", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta":{"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-02T00:03:00Z", "name":"Charlie", "age":42, "registration_date":"2023-12-23", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_overwrite_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_overwrite_raw.jsonl new file mode 100644 index 000000000000..144920cb7796 --- /dev/null +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_overwrite_raw.jsonl @@ -0,0 +1,4 @@ +// Only sync2 messages present in overwrite mode +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw.jsonl index 128a2d15bade..2fb3c2e010ad 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw.jsonl @@ -1,10 +1,10 @@ // We keep the records from the first sync -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}} -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes":[{"field":"address","change":"NULLED","reason":"SOURCE_RETRIEVAL_ERROR"}]}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} // And append the records from the second sync -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z", "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw2.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw2.jsonl index 6ae7bc9030ad..d9b153184fc1 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw2.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/dat/sync2_expectedrecords_raw2.jsonl @@ -1,2 +1,2 @@ -{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}, "_airbyte_meta": {"changes": []}} -{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different v2"}, "_airbyte_meta": {"changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:01.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} +{"_airbyte_extracted_at": "1970-01-01T00:00:02.000000Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different v2"}, "_airbyte_generation_id": 43, "_airbyte_meta": {"sync_id": 42, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_final.jsonl index 698c33c88769..89492e9429ce 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_final.jsonl @@ -1,7 +1,7 @@ -{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}} -{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": [{"field":"struct","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"array","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"number","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}} +{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 42, "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": 42, "changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}} +{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": [{"field":"struct","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"array","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"number","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}} // Note that for numbers where we parse the value to JSON (struct, array, unknown) we lose precision. // But for numbers where we create a NUMBER column, we do not lose precision (see the `number` column). -{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} +{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118, "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_raw.jsonl index 73351b7972a9..7d014b4c0a9c 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_expectedrecords_raw.jsonl @@ -1,5 +1,5 @@ -{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}} -{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}} -{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}, "_airbyte_meta": {"changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}} -{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}} -{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}} +{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}} +{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}} +{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}, "_airbyte_meta": {"sync_id": 42, "changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}} +{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}} +{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_generation_id": 42, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_final.jsonl index 0ea024ad6441..ecdbd9e9bac4 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_final.jsonl @@ -1,8 +1,8 @@ // Same as alltypes_expected but the meta didn't exist in v1 raw tables, so that information is not resurrected to the final. -{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": [{"field":"struct","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"array","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"number","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"integer","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"boolean","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"timestamp_with_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"timestamp_without_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"time_with_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"time_without_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"date","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}} +{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": [{"field":"struct","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"array","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"number","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"integer","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"boolean","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"timestamp_with_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"timestamp_without_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"time_with_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"time_without_timezone","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"},{"field":"date","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}} // Note that for numbers where we parse the value to JSON (struct, array, unknown) we lose precision. // But for numbers where we create a NUMBER column, we do not lose precision (see the `number` column). -{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} +{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118, "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_raw.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_raw.jsonl index a341d911fbbc..cff6ccdb7837 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_raw.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/alltypes_v1v2_expectedrecords_raw.jsonl @@ -1,5 +1,5 @@ -{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}} -{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}} -{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}} -{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}} -{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}} +{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}} +{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}} +{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}} +{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}} +{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_generation_id": 0, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/incrementaldedup_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/incrementaldedup_expectedrecords_final.jsonl index 47da4b3bceef..2424bebe7b6a 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/incrementaldedup_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/incrementaldedup_expectedrecords_final.jsonl @@ -1,2 +1,2 @@ -{"_airbyte_raw_id": "80c99b54-54b4-43bd-b51b-1f67dafa2c52", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00.000000Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84} -{"_airbyte_raw_id": "b9ac9f01-abc1-4e7c-89e5-eac9223d5726", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": [{"field":"integer","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}, "id1": 2, "id2": 100, "updated_at": "2023-01-01T03:00:01.000000Z", "string": "Bob"} +{"_airbyte_raw_id": "80c99b54-54b4-43bd-b51b-1f67dafa2c52", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00.000000Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84} +{"_airbyte_raw_id": "b9ac9f01-abc1-4e7c-89e5-eac9223d5726", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": [{"field":"integer","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}, "id1": 2, "id2": 100, "updated_at": "2023-01-01T03:00:01.000000Z", "string": "Bob"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/json_types_in_string_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/json_types_in_string_expectedrecords_final.jsonl index e015923deeb7..24c68af6eaff 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/json_types_in_string_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/json_types_in_string_expectedrecords_final.jsonl @@ -1,5 +1,5 @@ -{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "[\"I\",\"am\",\"an\",\"array\"]", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "{\"I\":\"am\",\"an\":\"object\"}", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "true", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "3.14", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} -{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "I am a valid json string", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} +{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "[\"I\",\"am\",\"an\",\"array\"]", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "{\"I\":\"am\",\"an\":\"object\"}", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "true", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "3.14", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} +{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "I am a valid json string", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/mixedcasecolumnname_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/mixedcasecolumnname_expectedrecords_final.jsonl index 3b494b78e4c4..d6b4bb964629 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/mixedcasecolumnname_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/mixedcasecolumnname_expectedrecords_final.jsonl @@ -1 +1 @@ -{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fce", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "id1": 6, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "iamacasesensitivecolumnname": "Case senstive value"} +{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fce", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "id1": 6, "id2": 100, "updated_at": "2023-01-01T01:00:00.000000Z", "iamacasesensitivecolumnname": "Case senstive value"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/nocolumns_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/nocolumns_expectedrecords_final.jsonl index d14bcddf132f..0f204d84fdc9 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/nocolumns_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/nocolumns_expectedrecords_final.jsonl @@ -1 +1 @@ -{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}} +{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/reservedkeywords_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/reservedkeywords_expectedrecords_final.jsonl index 8ffcc0c73bdc..b7e7144fe2b2 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/reservedkeywords_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/reservedkeywords_expectedrecords_final.jsonl @@ -1 +1 @@ -{"_airbyte_raw_id":"b2e0efc4-38a8-47ba-970c-8103f09f08d5","_airbyte_extracted_at":"2023-01-01T00:00:00.000000Z","_airbyte_meta":{"changes":[]}, "current_date": "foo", "join": "bar"} +{"_airbyte_raw_id":"b2e0efc4-38a8-47ba-970c-8103f09f08d5", "_airbyte_extracted_at":"2023-01-01T00:00:00.000000Z", "_airbyte_meta":{"sync_id": null, "changes":[]}, "current_date": "foo", "join": "bar"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/timestampformats_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/timestampformats_expectedrecords_final.jsonl index 33a87e1f4748..102066ffffc7 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/timestampformats_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/timestampformats_expectedrecords_final.jsonl @@ -2,15 +2,15 @@ // TIME, TIMETZ, TIMESTAMP, TIMESTAMPTZ values are UTC in user tables. // Note that redshift stores precision to microseconds. Java deserialization in tests preserves them only for non-zero values // except for timestamp with time zone where Z is required at end for even zero values -{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "time_with_timezone": "12:34:56Z"} -{"_airbyte_raw_id": "05028c5f-7813-4e9c-bd4b-387d1f8ba435", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} -{"_airbyte_raw_id": "95dfb0c6-6a67-4ba0-9935-643bebc90437", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} -{"_airbyte_raw_id": "f3d8abe2-bb0f-4caf-8ddc-0641df02f3a9", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} -{"_airbyte_raw_id": "a81ed40a-2a49-488d-9714-d53e8b052968", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} -{"_airbyte_raw_id": "c07763a0-89e6-4cb7-b7d0-7a34a7c9918a", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} -{"_airbyte_raw_id": "358d3b52-50ab-4e06-9094-039386f9bf0d", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} -{"_airbyte_raw_id": "db8200ac-b2b9-4b95-a053-8a0343042751", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56.123000Z", "time_with_timezone": "12:34:56.123Z"} +{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56.000000Z", "time_with_timezone": "12:34:56Z"} +{"_airbyte_raw_id": "05028c5f-7813-4e9c-bd4b-387d1f8ba435", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} +{"_airbyte_raw_id": "95dfb0c6-6a67-4ba0-9935-643bebc90437", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} +{"_airbyte_raw_id": "f3d8abe2-bb0f-4caf-8ddc-0641df02f3a9", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56.000000Z", "time_with_timezone": "20:34:56Z"} +{"_airbyte_raw_id": "a81ed40a-2a49-488d-9714-d53e8b052968", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} +{"_airbyte_raw_id": "c07763a0-89e6-4cb7-b7d0-7a34a7c9918a", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} +{"_airbyte_raw_id": "358d3b52-50ab-4e06-9094-039386f9bf0d", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56.000000Z", "time_with_timezone": "04:34:56Z"} +{"_airbyte_raw_id": "db8200ac-b2b9-4b95-a053-8a0343042751", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56.123000Z", "time_with_timezone": "12:34:56.123Z"} -{"_airbyte_raw_id": "10ce5d93-6923-4217-a46f-103833837038", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56", "time_without_timezone": "12:34:56", "date": "2023-01-23"} +{"_airbyte_raw_id": "10ce5d93-6923-4217-a46f-103833837038", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56", "time_without_timezone": "12:34:56", "date": "2023-01-23"} // Bigquery returns 6 decimal places if there are any decimal places... but not for timestamp_with_timezone -{"_airbyte_raw_id": "a7a6e176-7464-4a0b-b55c-b4f936e8d5a1", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56.123", "time_without_timezone": "12:34:56.123"} +{"_airbyte_raw_id": "a7a6e176-7464-4a0b-b55c-b4f936e8d5a1", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56.123", "time_without_timezone": "12:34:56.123"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/weirdcolumnnames_expectedrecords_final.jsonl b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/weirdcolumnnames_expectedrecords_final.jsonl index 9d73b0601264..540dc40d3235 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/weirdcolumnnames_expectedrecords_final.jsonl +++ b/airbyte-integrations/connectors/destination-redshift/src/test-integration/resources/sqlgenerator/weirdcolumnnames_expectedrecords_final.jsonl @@ -6,4 +6,4 @@ // * includes$$doubledollar -> includes__doubledollar // * includes.period -> includes_period // * endswithbackslash\ -> endswithbackslash_ -{"_airbyte_raw_id": "7e7330a1-42fb-41ec-a955-52f18bd61964", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00.000000Z", "_starts_with_dollar_sign": "foo", "includes_doublequote": "foo", "includes_singlequote": "foo", "includes_backtick": "foo", "includes_period": "foo", "includes__doubledollar": "foo", "endswithbackslash_": "foo"} +{"_airbyte_raw_id": "7e7330a1-42fb-41ec-a955-52f18bd61964", "_airbyte_extracted_at": "2023-01-01T00:00:00.000000Z", "_airbyte_meta": {"sync_id": null, "changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00.000000Z", "_starts_with_dollar_sign": "foo", "includes_doublequote": "foo", "includes_singlequote": "foo", "includes_backtick": "foo", "includes_period": "foo", "includes__doubledollar": "foo", "endswithbackslash_": "foo"} diff --git a/airbyte-integrations/connectors/destination-redshift/src/test/resources/typing_deduping_with_cdc.sql b/airbyte-integrations/connectors/destination-redshift/src/test/resources/typing_deduping_with_cdc.sql index e224ff77babe..591b75a1e617 100644 --- a/airbyte-integrations/connectors/destination-redshift/src/test/resources/typing_deduping_with_cdc.sql +++ b/airbyte-integrations/connectors/destination-redshift/src/test/resources/typing_deduping_with_cdc.sql @@ -18,6 +18,7 @@ insert into "test_schema"."users_finalunittest" ( "_ab_cdc_deleted_at", "_airbyte_raw_id", "_airbyte_extracted_at", + "_airbyte_generation_id", "_airbyte_meta" ) with @@ -44,6 +45,7 @@ with cast("_airbyte_data"."_ab_cdc_deleted_at" as timestamp with time zone) as "_ab_cdc_deleted_at", "_airbyte_raw_id", "_airbyte_extracted_at", + "_airbyte_generation_id", OBJECT( 'changes', ARRAY_CONCAT( @@ -147,7 +149,9 @@ with and "_airbyte_meta"."changes" is not null and IS_ARRAY("_airbyte_meta"."changes") ) THEN "_airbyte_meta"."changes" ELSE ARRAY() END - ) + ), + 'sync_id', + "_airbyte_meta"."sync_id" ) as "_airbyte_meta" from "test_schema"."users_raw" where ( @@ -191,6 +195,7 @@ select "_ab_cdc_deleted_at", "_airbyte_raw_id", "_airbyte_extracted_at", + "_airbyte_generation_id", "_airbyte_meta" from "numbered_rows" where "row_number" = 1; diff --git a/docs/integrations/destinations/redshift.md b/docs/integrations/destinations/redshift.md index 637c1017b8e3..3404930e527e 100644 --- a/docs/integrations/destinations/redshift.md +++ b/docs/integrations/destinations/redshift.md @@ -244,6 +244,7 @@ Each stream will be output into its own raw table in Redshift. Each table will c | Version | Date | Pull Request | Subject | |:--------|:-----------|:-----------------------------------------------------------|:-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| +| 3.2.0 | 2024-07-02 | [40201](https://github.com/airbytehq/airbyte/pull/40201) | Add `_airbyte_generation_id` column, and add `sync_id` to `_airbyte_meta` column | | 3.1.1 | 2024-06-26 | [39008](https://github.com/airbytehq/airbyte/pull/39008) | Internal code changes | | 3.1.0 | 2024-06-26 | [39141](https://github.com/airbytehq/airbyte/pull/39141) | Remove nonfunctional "encrypted staging" option | | 3.0.0 | 2024-06-04 | [38886](https://github.com/airbytehq/airbyte/pull/38886) | Remove standard inserts mode |