From 2ce873a1e5a1d8a9f6ea0d672be11e5aa34faef6 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 10 Apr 2025 15:23:34 -0700 Subject: [PATCH 01/55] stashing WIP --- .../util/BigqueryDestinationHandlerFactory.kt | 17 +++ .../util/BigquerySqlGeneratorFactory.kt | 13 ++ .../bigquery/util/ParsedCatalogFactory.kt | 27 +++++ .../bigquery/write/BigqueryStreamLoader.kt | 9 ++ .../bigquery/write/BigqueryWriter.kt | 112 ++++++++++++++++++ 5 files changed, 178 insertions(+) create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt new file mode 100644 index 0000000000000..8f2b328f56e97 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt @@ -0,0 +1,17 @@ +package io.airbyte.integrations.destination.bigquery.util + +import com.google.cloud.bigquery.BigQuery +import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler +import io.micronaut.context.annotation.Factory +import jakarta.inject.Singleton + +@Factory +class BigqueryDestinationHandlerFactory( + private val bigquery: BigQuery, + private val config: BigqueryConfiguration, +) { + @Singleton + fun make(): BigQueryDestinationHandler = + BigQueryDestinationHandler(bigquery, config.datasetLocation.region) +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt new file mode 100644 index 0000000000000..0d098c842ee8e --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt @@ -0,0 +1,13 @@ +package io.airbyte.integrations.destination.bigquery.util + +import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator +import io.micronaut.context.annotation.Factory +import jakarta.inject.Singleton + +@Factory +class BigquerySqlGeneratorFactory(private val config: BigqueryConfiguration) { + @Singleton + fun make(): BigQuerySqlGenerator = + BigQuerySqlGenerator(config.projectId, config.datasetLocation.region) +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt new file mode 100644 index 0000000000000..063813b918ba2 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt @@ -0,0 +1,27 @@ +package io.airbyte.integrations.destination.bigquery.util + +import io.airbyte.cdk.load.command.DestinationCatalog +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser +import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog +import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator +import io.micronaut.context.annotation.Factory +import jakarta.inject.Singleton + +// TODO this is... probably a temporary thing? +// probably should replace it with better concepts in the bulk CDK +@Factory +class ParsedCatalogFactory( + private val catalog: DestinationCatalog, + private val config: BigqueryConfiguration, + private val sqlGenerator: BigQuerySqlGenerator, +) { + @Singleton + fun make(): ParsedCatalog = + CatalogParser( + sqlGenerator, + defaultNamespace = config.datasetId, + rawNamespace = config.rawTableDataset, + ) + .parseCatalog(catalog.asProtocolObject()) +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt index 4423b4c60e29a..5ad7e0066fc8c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt @@ -9,6 +9,7 @@ import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.write.StreamLoader import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter +import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.github.oshai.kotlinlogging.KotlinLogging @@ -21,6 +22,14 @@ class BigqueryStreamLoader( ) : StreamLoader { override suspend fun start() { super.start() + + BigQueryDirectLoadingStorageOperation( + bigquery, + bigQueryClientChunkSize = TODO(), + BigQueryRecordFormatter(), + + ) + logger.info { "Creating dataset if needed: ${config.rawTableDataset}" } BigQueryUtils.getOrCreateDataset( bigquery, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index f24c9e336678b..d9385ec9071b9 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -6,18 +6,130 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.TableId +import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.V2_WITH_GENERATION +import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig +import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer +import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations +import io.airbyte.cdk.integrations.destination.gcs.credential.GcsCredentialConfig +import io.airbyte.cdk.integrations.destination.gcs.credential.GcsHmacKeyCredentialConfig +import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat +import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations +import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader +import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation +import io.airbyte.integrations.base.destination.operation.StandardStreamOperation +import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser +import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus +import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog import io.airbyte.integrations.base.destination.typing_deduping.StreamId +import io.airbyte.integrations.destination.bigquery.BigQueryUtils +import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter +import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDV2Migration +import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState +import io.airbyte.integrations.destination.bigquery.migrators.BigqueryAirbyteMetaAndGenerationIdMigration +import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation +import io.airbyte.integrations.destination.bigquery.operation.BigQueryGcsStorageOperation +import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration +import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing +import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator import jakarta.inject.Singleton @Singleton class BigqueryWriter( private val bigquery: BigQuery, private val config: BigqueryConfiguration, + private val sqlGenerator: BigQuerySqlGenerator, + private val parsedCatalog: ParsedCatalog, + private val destinationHandler: BigQueryDestinationHandler, ) : DestinationWriter { + private lateinit var syncOperation: DefaultSyncOperation + + override suspend fun setup() { + super.setup() + + val migrations = + listOf( + BigQueryDV2Migration(sqlGenerator, bigquery), + BigqueryAirbyteMetaAndGenerationIdMigration(bigquery), + ) + syncOperation = + when (config.loadingMethod) { + is BatchedStandardInsertConfiguration -> { + val bigQueryLoadingStorageOperation = + BigQueryDirectLoadingStorageOperation( + bigquery, + bigQueryClientChunkSize = null, + BigQueryRecordFormatter(), + sqlGenerator, + destinationHandler, + config.datasetLocation.region, + ) + DefaultSyncOperation( + parsedCatalog, + destinationHandler, + config.datasetId, + { initialStatus: DestinationInitialStatus, disableTD + -> + StandardStreamOperation( + bigQueryLoadingStorageOperation, + initialStatus, + disableTD, + ) + }, + migrations, + config.disableTypingDeduping, + ) + } + is GcsStagingConfiguration -> { + val gcsNameTransformer = GcsNameTransformer() + // TODO validate this + val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) + GcsDestinationConfig( + config.loadingMethod.gcsClientConfig.gcsBucketName, + bucketPath = config.loadingMethod.gcsClientConfig.path, + bucketRegion = config.datasetLocation.region, + gcsCredentialConfig = GcsHmacKeyCredentialConfig(), + formatConfig = TODO(), + ) + val keepStagingFiles = + config.loadingMethod.filePostProcessing == GcsFilePostProcessing.KEEP + val gcsOperations = + GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) + val bigQueryGcsStorageOperations = + BigQueryGcsStorageOperation( + gcsOperations, + gcsConfig, + gcsNameTransformer, + keepStagingFiles, + bigquery, + sqlGenerator, + destinationHandler, + ) + DefaultSyncOperation( + parsedCatalog, + destinationHandler, + config.datasetId, + { initialStatus: DestinationInitialStatus, disableTD -> + StagingStreamOperations( + bigQueryGcsStorageOperations, + initialStatus, + FileUploadFormat.CSV, + V2_WITH_GENERATION, + disableTD, + ) + }, + migrations, + config.disableTypingDeduping, + ) + } + } + } + override fun createStreamLoader(stream: DestinationStream): StreamLoader { return BigqueryStreamLoader(stream, bigquery, config) } From 6e01e976749da321cd54cb8a08386c3bf7168e77 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 10:39:09 -0700 Subject: [PATCH 02/55] stashing WIP --- .../cdk/load/orchestration/DbStreamLoader.kt | 13 ++ .../cdk/load/orchestration/DbWriter.kt | 15 +++ .../load/orchestration/DestinationHandler.kt | 13 ++ .../load/orchestration/DestinationNames.kt | 30 +++++ .../DestinationStatusGatherer.kt | 11 ++ .../io/airbyte/cdk/load/orchestration/Sql.kt | 111 ++++++++++++++++++ .../LegacyTypingDedupingStreamLoader.kt | 49 ++++++++ .../TypingDedupingDestinationInitialStatus.kt | 8 ++ .../TypingDedupingFinalTableOperations.kt | 50 ++++++++ .../TypingDedupingRawTableOperations.kt | 43 +++++++ .../TypingDedupingSqlGenerator.kt | 4 + .../TypingDedupingWriter.kt | 46 ++++++++ 12 files changed, 393 insertions(+) create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt new file mode 100644 index 0000000000000..d9fe1be1fe212 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt @@ -0,0 +1,13 @@ +package io.airbyte.cdk.load.orchestration + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.state.StreamProcessingFailed +import io.airbyte.cdk.load.write.StreamLoader + +class DbStreamLoader(override val stream: DestinationStream) : StreamLoader { + override suspend fun start() { + } + + override suspend fun close(streamFailure: StreamProcessingFailed?) { + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt new file mode 100644 index 0000000000000..126e6224f7fcb --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt @@ -0,0 +1,15 @@ +package io.airbyte.cdk.load.orchestration + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.write.DestinationWriter +import io.airbyte.cdk.load.write.StreamLoader + +class DbWriter : DestinationWriter { + override suspend fun setup() { + // create all namespaces + } + + override fun createStreamLoader(stream: DestinationStream): StreamLoader { + TODO("Not yet implemented") + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt new file mode 100644 index 0000000000000..856b86a5a83f4 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt @@ -0,0 +1,13 @@ +package io.airbyte.cdk.load.orchestration + +interface DestinationHandler { + fun execute(sql: Sql) + + /** + * Create the namespaces (typically something like `create schema`). + * + * This function should assume that all `namespaces` are valid identifiers, i.e. any special + * characters have already been escaped, they respect identifier name length, etc. + */ + fun createNamespaces(namespaces: List) +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt new file mode 100644 index 0000000000000..03b67ee70d691 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -0,0 +1,30 @@ +package io.airbyte.cdk.load.orchestration + +data class TableNames( + // this is pretty dumb, but in theory we could have: + // * old-style implementation: raw+final tables both exist + // * only the raw table exists (i.e. T+D disabled) + // * only the final table exists (i.e. new-style direct-load tables) + val rawTableName: TableName?, + val finalTableName: TableName?, +) { + init { + check(rawTableName != null || finalTableName != null) { + "At least one table name should be nonnull" + } + } + companion object { + // TODO comment explaining this + const val TMP_TABLE_SUFFIX = "_airbyte_tmp" + } +} + +data class TableName(val namespace: String, val name: String) + +/** + * map from the column name as declared in the schema, + * to the column name that we'll create in the final (typed) table. + */ +@JvmInline +value class ColumnNameMapping(private val columnNameMapping: Map) : + Map by columnNameMapping diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt new file mode 100644 index 0000000000000..115cac64bf673 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt @@ -0,0 +1,11 @@ +package io.airbyte.cdk.load.orchestration + +import io.airbyte.cdk.load.command.DestinationStream + +interface DestinationInitialStatus + +fun interface DestinationStatusGatherer { + fun gatherInitialStatus( + streams: Map>, + ): Map +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt new file mode 100644 index 0000000000000..c857d81dcd092 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt @@ -0,0 +1,111 @@ +package io.airbyte.cdk.load.orchestration + +import java.util.function.Consumer + +/** + * Represents a list of SQL transactions, where each transaction consists of one or more SQL + * statements. Each transaction MUST NOT contain the BEGIN/COMMIT statements. Each inner list is a + * single transaction, and each String is a single statement within that transaction. + * + * Most callers likely only need a single transaction, but e.g. BigQuery disallows running DDL + * inside transactions, and so needs to run sequential "CREATE SCHEMA", "CREATE TABLE" as separate + * transactions. + * + * Callers are encouraged to use the static factory methods instead of the public constructor. + */ +data class Sql(val transactions: List>) { + /** + * @param begin The SQL statement to start a transaction, typically "BEGIN" + * @param commit The SQL statement to commit a transaction, typically "COMMIT" + * @return A list of SQL strings, each of which represents a transaction. + */ + fun asSqlStrings(begin: String, commit: String): List { + return transactions.map { transaction: List -> + // If there's only one statement, we don't need to wrap it in a transaction. + if (transaction.size == 1) { + return@map transaction[0] + } + val builder = StringBuilder() + builder.append(begin) + builder.append(";\n") + transaction.forEach { + builder.append(it) + // No semicolon - statements already end with a semicolon + builder.append("\n") + } + builder.append(commit) + builder.append(";\n") + builder.toString() + } + } + + init { + transactions.forEach( + Consumer { transaction: List -> + require(transaction.isNotEmpty()) { "Transaction must not be empty" } + require(!transaction.any { it.isEmpty() }) { + "Transaction must not contain empty statements" + } + } + ) + } + + companion object { + /** Execute a list of SQL statements in a single transaction. */ + fun transactionally(statements: List): Sql { + return create(listOf(statements)) + } + + fun transactionally(vararg statements: String): Sql { + return transactionally(listOf(*statements)) + } + + /** Execute each statement as its own transaction. */ + fun separately(statements: List): Sql { + return create(statements.map { listOf(it) }) + } + + fun separately(vararg statements: String): Sql { + return separately(statements.asList()) + } + + /** + * Convenience method for indicating intent. Equivalent to calling [transactionally] or + * [.separately] with the same string. + */ + fun of(statement: String): Sql { + return transactionally(statement) + } + + fun concat(vararg sqls: Sql): Sql { + return create(sqls.flatMap { it.transactions }) + } + + fun concat(sqls: List): Sql { + return create(sqls.flatMap { it.transactions }) + } + + /** + * Utility method to create a Sql object without empty statements/transactions, and + * appending semicolons when needed. + */ + fun create(transactions: List>): Sql { + return Sql( + transactions + .map { transaction: List -> + transaction + .map { it.trim() } + .filter { it.isNotEmpty() } + .map { + if (!it.endsWith(";")) { + "$it;" + } else { + it + } + } + } + .filter { it.isNotEmpty() } + ) + } + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt new file mode 100644 index 0000000000000..1387214c00e07 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt @@ -0,0 +1,49 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.state.StreamProcessingFailed +import io.airbyte.cdk.load.write.StreamLoader + +class LegacyTypingDedupingStreamLoader( + override val stream: DestinationStream, + private val initialStatus: TypingDedupingDestinationInitialStatus, + private val rawTableName: TableName, + private val finalTableName: TableName, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, +) : StreamLoader { + + override suspend fun start() { + // TODO do all the truncate stuff + rawTableOperations.prepareRawTable( + rawTableName, + suffix = "", + replace = false, + ) + finalTableOperations.createFinalTable( + stream, + finalTableName, + suffix = "", + replace = false, + ) + } + + override suspend fun close(streamFailure: StreamProcessingFailed?) { + // TODO is this handled in the bulk load pipeline already? + rawTableOperations.cleanupStage(rawTableName) + if (streamFailure == null) { + // TODO only do this in truncate mode, do all the correct truncate stuff + rawTableOperations.overwriteRawTable(rawTableName, suffix = "_airbyte_tmp") + finalTableOperations.typeAndDedupe( + stream, + rawTableName, + finalTableName, + maxProcessedTimestamp = TODO(), + finalTableSuffix = "", + ) + // TODO extract constant for suffix + finalTableOperations.overwriteFinalTable(finalTableName, suffix = "") + } + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt new file mode 100644 index 0000000000000..7c1c9a4a42d78 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -0,0 +1,8 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.orchestration.DestinationInitialStatus + +data class TypingDedupingDestinationInitialStatus( + val isFinalTablePresent: Boolean, + // TODO other stuff +) : DestinationInitialStatus diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt new file mode 100644 index 0000000000000..4640b4609d745 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -0,0 +1,50 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.DestinationHandler +import io.airbyte.cdk.load.orchestration.TableName +import java.time.Instant + +class TypingDedupingFinalTableOperations( + private val sqlGenerator: TypingDedupingSqlGenerator, + private val destinationHandler: DestinationHandler, +) { + fun createFinalTable( + stream: DestinationStream, + finalTableName: TableName, + suffix: String, + replace: Boolean + ) { + destinationHandler.execute(TODO()) + } + + /** Reset the final table using a temp table or ALTER existing table's columns. */ + fun softResetFinalTable( + stream: DestinationStream, + finalTableName: TableName, + ) { + destinationHandler.execute(TODO()) + } + + /** + * Attempt to atomically swap the final table from the temp version. + * This could be destination specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE + * ... SELECT *, DROP TABLE + */ + fun overwriteFinalTable( + finalTableName: TableName, + suffix: String, + ) { + TODO() + } + + fun typeAndDedupe( + stream: DestinationStream, + rawTableName: TableName, + finalTableName: TableName, + maxProcessedTimestamp: Instant?, + finalTableSuffix: String + ) { + TODO() + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt new file mode 100644 index 0000000000000..68716a796da96 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt @@ -0,0 +1,43 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.orchestration.TableName + +interface TypingDedupingRawTableOperations { + /** + * Prepare the raw table, including any associated blob storage. + * Similar to [createFinalTable], accepts a [suffix] parameter, which should be used in + * conjunction with [overwriteRawTable]. + * + * @param replace If true, then replace existing resources with empty e.g. tables. If false, + * then leave existing resources untouched. + */ + fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean = false) + + /** + * Swap the "temporary" raw table into the "real" raw table. For example, `DROP TABLE IF NOT EXISTS + * airbyte_internal.foo; ALTER TABLE airbyte_internal.foo_tmp RENAME TO foo`. + */ + fun overwriteRawTable(rawTableName: TableName, suffix: String) + + /** + * Copy all records from the temporary raw table into the real raw table, then drop the temporary raw table. + * For example `INSERT INTO airbyte_internal.foo SELECT * FROM airbyte_internal.foo_tmp; DROP + * TABLE airbyte_internal.foo_tmp`. + */ + fun transferFromTempRawTable(rawTableName: TableName, suffix: String) + + /** + * Get the generation of a single record in the raw table. Not necessarily the min or max + * generation, just _any_ record. + * + * [LegacyTypingDedupingStreamLoader] is responsible for orchestrating the raw tables so that + * the temp raw table always contains exactly one generation. + * + * @return The generation ID of a record in the raw table, or `null` if the raw table is empty. + */ + fun getRawTableGeneration(rawTableName: TableName, suffix: String): Long? + + // TODO this probably could be named better + /** Delete all staged data from blob storage. */ + fun cleanupStage(rawTableName: TableName) +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt new file mode 100644 index 0000000000000..4b5c77df9b96a --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -0,0 +1,4 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +interface TypingDedupingSqlGenerator { +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt new file mode 100644 index 0000000000000..71ed3ff6761c1 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -0,0 +1,46 @@ +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.command.DestinationCatalog +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.DestinationStatusGatherer +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.write.DestinationWriter +import io.airbyte.cdk.load.write.StreamLoader + +class TypingDedupingWriter( + private val catalog: DestinationCatalog, + private val stateGatherer: DestinationStatusGatherer, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, +) : DestinationWriter { + private lateinit var names: Map> + private lateinit var initialStatuses: Map + + override suspend fun setup() { + // TODO + // -1. figure out table/column names + // 0. gather state + // 1. execute migrations + // 2. soft reset + // 3. gather state + names = TODO() + val initialInitialStatuses: Map = stateGatherer.gatherInitialStatus(names) + // TODO migrations + // TODO soft reset if needed + // TODO only refetch streams that need to be refetched + initialStatuses = stateGatherer.gatherInitialStatus(names) + } + + override fun createStreamLoader(stream: DestinationStream): StreamLoader { + val tableNames = names[stream]!!.first + return LegacyTypingDedupingStreamLoader( + stream, + initialStatuses[stream.descriptor]!!, + tableNames.rawTableName!!, + tableNames.finalTableName!!, + rawTableOperations, + finalTableOperations, + ) + } +} From 16c3cfd7dad2680857b002d3432ccb2da0b4f4de Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 12:59:38 -0700 Subject: [PATCH 03/55] stashing WIP --- .../cdk/load/orchestration/DbStreamLoader.kt | 6 ++-- .../load/orchestration/DestinationNames.kt | 4 +-- .../DestinationStatusGatherer.kt | 6 +++- .../TypingDedupingDestinationInitialStatus.kt | 34 +++++++++++++++++-- .../TypingDedupingFinalTableOperations.kt | 5 ++- .../TypingDedupingRawTableOperations.kt | 15 ++++---- .../TypingDedupingSqlGenerator.kt | 3 +- .../TypingDedupingWriter.kt | 12 +++++-- .../bigquery/write/BigqueryStreamLoader.kt | 1 - .../bigquery/write/BigqueryWriter.kt | 12 +++---- 10 files changed, 66 insertions(+), 32 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt index d9fe1be1fe212..26a51492ecb08 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt @@ -5,9 +5,7 @@ import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader class DbStreamLoader(override val stream: DestinationStream) : StreamLoader { - override suspend fun start() { - } + override suspend fun start() {} - override suspend fun close(streamFailure: StreamProcessingFailed?) { - } + override suspend fun close(streamFailure: StreamProcessingFailed?) {} } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index 03b67ee70d691..9af3a0f8c0003 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -22,8 +22,8 @@ data class TableNames( data class TableName(val namespace: String, val name: String) /** - * map from the column name as declared in the schema, - * to the column name that we'll create in the final (typed) table. + * map from the column name as declared in the schema, to the column name that we'll create in the + * final (typed) table. */ @JvmInline value class ColumnNameMapping(private val columnNameMapping: Map) : diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt index 115cac64bf673..1369b3d2eee8d 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration import io.airbyte.cdk.load.command.DestinationStream @@ -7,5 +11,5 @@ interface DestinationInitialStatus fun interface DestinationStatusGatherer { fun gatherInitialStatus( streams: Map>, - ): Map + ): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt index 7c1c9a4a42d78..98b4d8085cdc5 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -1,8 +1,38 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.orchestration.DestinationInitialStatus +import java.time.Instant data class TypingDedupingDestinationInitialStatus( - val isFinalTablePresent: Boolean, - // TODO other stuff + /** + * Initial status of the final table, or null if the table doesn't exist yet. + */ + val finalTableStatus: FinalTableInitialStatus?, + val rawTableStatus: RawTableInitialStatus?, + val tempRawTableStatus: RawTableInitialStatus?, ) : DestinationInitialStatus + +data class FinalTableInitialStatus( + val isSchemaMismatch: Boolean, + val isEmpty: Boolean, + /** + * The generation ID of _any_ record from the final table, or `null` if the table is empty. + */ + val finalTableGenerationId: Long?, +) + +data class RawTableInitialStatus( + /** + * Whether there were any records with null `_airbyte_loaded_at`, at the time that this status + * was fetched. + */ + val hasUnprocessedRecords: Boolean, + /** + * The highest timestamp such that all records in `SELECT * FROM raw_table WHERE + * _airbyte_extracted_at <= ?` have a nonnull `_airbyte_loaded_at`. + * + * Destinations MAY use this value to only run T+D on records with `_airbyte_extracted_at > ?` + * (note the strictly-greater comparison). + */ + val maxProcessedTimestamp: Instant?, +) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 4640b4609d745..1c86986204a5e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -27,9 +27,8 @@ class TypingDedupingFinalTableOperations( } /** - * Attempt to atomically swap the final table from the temp version. - * This could be destination specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE - * ... SELECT *, DROP TABLE + * Attempt to atomically swap the final table from the temp version. This could be destination + * specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE ... SELECT *, DROP TABLE */ fun overwriteFinalTable( finalTableName: TableName, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt index 68716a796da96..28f3e18571583 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt @@ -4,9 +4,8 @@ import io.airbyte.cdk.load.orchestration.TableName interface TypingDedupingRawTableOperations { /** - * Prepare the raw table, including any associated blob storage. - * Similar to [createFinalTable], accepts a [suffix] parameter, which should be used in - * conjunction with [overwriteRawTable]. + * Prepare the raw table, including any associated blob storage. Similar to [createFinalTable], + * accepts a [suffix] parameter, which should be used in conjunction with [overwriteRawTable]. * * @param replace If true, then replace existing resources with empty e.g. tables. If false, * then leave existing resources untouched. @@ -14,15 +13,15 @@ interface TypingDedupingRawTableOperations { fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean = false) /** - * Swap the "temporary" raw table into the "real" raw table. For example, `DROP TABLE IF NOT EXISTS - * airbyte_internal.foo; ALTER TABLE airbyte_internal.foo_tmp RENAME TO foo`. + * Swap the "temporary" raw table into the "real" raw table. For example, `DROP TABLE IF NOT + * EXISTS airbyte_internal.foo; ALTER TABLE airbyte_internal.foo_tmp RENAME TO foo`. */ fun overwriteRawTable(rawTableName: TableName, suffix: String) /** - * Copy all records from the temporary raw table into the real raw table, then drop the temporary raw table. - * For example `INSERT INTO airbyte_internal.foo SELECT * FROM airbyte_internal.foo_tmp; DROP - * TABLE airbyte_internal.foo_tmp`. + * Copy all records from the temporary raw table into the real raw table, then drop the + * temporary raw table. For example `INSERT INTO airbyte_internal.foo SELECT * FROM + * airbyte_internal.foo_tmp; DROP TABLE airbyte_internal.foo_tmp`. */ fun transferFromTempRawTable(rawTableName: TableName, suffix: String) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index 4b5c77df9b96a..c081318ffbd3c 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -1,4 +1,3 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping -interface TypingDedupingSqlGenerator { -} +interface TypingDedupingSqlGenerator {} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 71ed3ff6761c1..575ead8f4dbb1 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationCatalog @@ -15,7 +19,8 @@ class TypingDedupingWriter( private val finalTableOperations: TypingDedupingFinalTableOperations, ) : DestinationWriter { private lateinit var names: Map> - private lateinit var initialStatuses: Map + private lateinit var initialStatuses: + Map override suspend fun setup() { // TODO @@ -25,7 +30,8 @@ class TypingDedupingWriter( // 2. soft reset // 3. gather state names = TODO() - val initialInitialStatuses: Map = stateGatherer.gatherInitialStatus(names) + val initialInitialStatuses: Map = + stateGatherer.gatherInitialStatus(names) // TODO migrations // TODO soft reset if needed // TODO only refetch streams that need to be refetched @@ -36,7 +42,7 @@ class TypingDedupingWriter( val tableNames = names[stream]!!.first return LegacyTypingDedupingStreamLoader( stream, - initialStatuses[stream.descriptor]!!, + initialStatuses[stream]!!, tableNames.rawTableName!!, tableNames.finalTableName!!, rawTableOperations, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt index 5ad7e0066fc8c..1cb99a1653c57 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt @@ -27,7 +27,6 @@ class BigqueryStreamLoader( bigquery, bigQueryClientChunkSize = TODO(), BigQueryRecordFormatter(), - ) logger.info { "Creating dataset if needed: ${config.rawTableDataset}" } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index d9385ec9071b9..fcbb7139a91c1 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -10,17 +10,14 @@ import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.V2_ import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations -import io.airbyte.cdk.integrations.destination.gcs.credential.GcsCredentialConfig import io.airbyte.cdk.integrations.destination.gcs.credential.GcsHmacKeyCredentialConfig import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations -import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation import io.airbyte.integrations.base.destination.operation.StandardStreamOperation -import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog import io.airbyte.integrations.base.destination.typing_deduping.StreamId @@ -73,8 +70,9 @@ class BigqueryWriter( parsedCatalog, destinationHandler, config.datasetId, - { initialStatus: DestinationInitialStatus, disableTD - -> + { + initialStatus: DestinationInitialStatus, + disableTD -> StandardStreamOperation( bigQueryLoadingStorageOperation, initialStatus, @@ -114,7 +112,9 @@ class BigqueryWriter( parsedCatalog, destinationHandler, config.datasetId, - { initialStatus: DestinationInitialStatus, disableTD -> + { + initialStatus: DestinationInitialStatus, + disableTD -> StagingStreamOperations( bigQueryGcsStorageOperations, initialStatus, From 0509cf71d9fe419f81c2e839d62be3582aeee68d Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 14:00:13 -0700 Subject: [PATCH 04/55] stashing WIP --- ...kt => DestinationInitialStatusGatherer.kt} | 2 +- .../TypingDedupingWriter.kt | 4 +- .../bigquery/util/ParsedCatalogFactory.kt | 27 ----- .../write/BigqueryInitialStateGatherer.kt | 13 +++ .../write/BigqueryRawTableOperations.kt | 26 +++++ .../bigquery/write/BigqueryStreamLoader.kt | 48 -------- .../bigquery/write/BigqueryWriter.kt | 110 +++--------------- 7 files changed, 58 insertions(+), 172 deletions(-) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{DestinationStatusGatherer.kt => DestinationInitialStatusGatherer.kt} (79%) delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt similarity index 79% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt index 1369b3d2eee8d..3dd9f0bf16d00 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt @@ -8,7 +8,7 @@ import io.airbyte.cdk.load.command.DestinationStream interface DestinationInitialStatus -fun interface DestinationStatusGatherer { +fun interface DestinationInitialStatusGatherer { fun gatherInitialStatus( streams: Map>, ): Map diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 575ead8f4dbb1..cf934c1d3d0ba 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -7,14 +7,14 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.DestinationStatusGatherer +import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader class TypingDedupingWriter( private val catalog: DestinationCatalog, - private val stateGatherer: DestinationStatusGatherer, + private val stateGatherer: DestinationInitialStatusGatherer, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, ) : DestinationWriter { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt deleted file mode 100644 index 063813b918ba2..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/ParsedCatalogFactory.kt +++ /dev/null @@ -1,27 +0,0 @@ -package io.airbyte.integrations.destination.bigquery.util - -import io.airbyte.cdk.load.command.DestinationCatalog -import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser -import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog -import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.micronaut.context.annotation.Factory -import jakarta.inject.Singleton - -// TODO this is... probably a temporary thing? -// probably should replace it with better concepts in the bulk CDK -@Factory -class ParsedCatalogFactory( - private val catalog: DestinationCatalog, - private val config: BigqueryConfiguration, - private val sqlGenerator: BigQuerySqlGenerator, -) { - @Singleton - fun make(): ParsedCatalog = - CatalogParser( - sqlGenerator, - defaultNamespace = config.datasetId, - rawNamespace = config.rawTableDataset, - ) - .parseCatalog(catalog.asProtocolObject()) -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt new file mode 100644 index 0000000000000..4e79b18b1ac99 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt @@ -0,0 +1,13 @@ +package io.airbyte.integrations.destination.bigquery.write + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus + +class BigqueryInitialStateGatherer : DestinationInitialStatusGatherer { + override fun gatherInitialStatus(streams: Map>): Map { + TODO("Not yet implemented") + } +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt new file mode 100644 index 0000000000000..ba9cfb5a5102b --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt @@ -0,0 +1,26 @@ +package io.airbyte.integrations.destination.bigquery.write + +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingRawTableOperations + +class BigqueryRawTableOperations : TypingDedupingRawTableOperations { + override fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean) { + TODO("Not yet implemented") + } + + override fun overwriteRawTable(rawTableName: TableName, suffix: String) { + TODO("Not yet implemented") + } + + override fun transferFromTempRawTable(rawTableName: TableName, suffix: String) { + TODO("Not yet implemented") + } + + override fun getRawTableGeneration(rawTableName: TableName, suffix: String): Long? { + TODO("Not yet implemented") + } + + override fun cleanupStage(rawTableName: TableName) { + TODO("Not yet implemented") + } +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt deleted file mode 100644 index 1cb99a1653c57..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryStreamLoader.kt +++ /dev/null @@ -1,48 +0,0 @@ -/* - * Copyright (c) 2025 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.write - -import com.google.cloud.bigquery.BigQuery -import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.write.StreamLoader -import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation -import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.github.oshai.kotlinlogging.KotlinLogging - -private val logger = KotlinLogging.logger {} - -class BigqueryStreamLoader( - override val stream: DestinationStream, - private val bigquery: BigQuery, - private val config: BigqueryConfiguration, -) : StreamLoader { - override suspend fun start() { - super.start() - - BigQueryDirectLoadingStorageOperation( - bigquery, - bigQueryClientChunkSize = TODO(), - BigQueryRecordFormatter(), - ) - - logger.info { "Creating dataset if needed: ${config.rawTableDataset}" } - BigQueryUtils.getOrCreateDataset( - bigquery, - config.rawTableDataset, - config.datasetLocation.region - ) - // TODO also need to create final table dataset - logger.info { - "Creating table if needed: ${TempUtils.rawTableId(config, stream.descriptor)}" - } - BigQueryUtils.createPartitionedTableIfNotExists( - bigquery, - TempUtils.rawTableId(config, stream.descriptor), - BigQueryRecordFormatter.SCHEMA_V2, - ) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index fcbb7139a91c1..4fb0fe79ad9e6 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -13,7 +13,10 @@ import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations import io.airbyte.cdk.integrations.destination.gcs.credential.GcsHmacKeyCredentialConfig import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations +import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingFinalTableOperations +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingWriter import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation @@ -34,110 +37,29 @@ import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator +import io.micronaut.context.annotation.Factory import jakarta.inject.Singleton -@Singleton -class BigqueryWriter( +@Factory +class BigqueryWriterFactory( + private val catalog: DestinationCatalog, private val bigquery: BigQuery, private val config: BigqueryConfiguration, private val sqlGenerator: BigQuerySqlGenerator, - private val parsedCatalog: ParsedCatalog, private val destinationHandler: BigQueryDestinationHandler, -) : DestinationWriter { - private lateinit var syncOperation: DefaultSyncOperation - - override suspend fun setup() { - super.setup() - - val migrations = - listOf( - BigQueryDV2Migration(sqlGenerator, bigquery), - BigqueryAirbyteMetaAndGenerationIdMigration(bigquery), - ) - syncOperation = - when (config.loadingMethod) { - is BatchedStandardInsertConfiguration -> { - val bigQueryLoadingStorageOperation = - BigQueryDirectLoadingStorageOperation( - bigquery, - bigQueryClientChunkSize = null, - BigQueryRecordFormatter(), - sqlGenerator, - destinationHandler, - config.datasetLocation.region, - ) - DefaultSyncOperation( - parsedCatalog, - destinationHandler, - config.datasetId, - { - initialStatus: DestinationInitialStatus, - disableTD -> - StandardStreamOperation( - bigQueryLoadingStorageOperation, - initialStatus, - disableTD, - ) - }, - migrations, - config.disableTypingDeduping, - ) - } - is GcsStagingConfiguration -> { - val gcsNameTransformer = GcsNameTransformer() - // TODO validate this - val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) - GcsDestinationConfig( - config.loadingMethod.gcsClientConfig.gcsBucketName, - bucketPath = config.loadingMethod.gcsClientConfig.path, - bucketRegion = config.datasetLocation.region, - gcsCredentialConfig = GcsHmacKeyCredentialConfig(), - formatConfig = TODO(), - ) - val keepStagingFiles = - config.loadingMethod.filePostProcessing == GcsFilePostProcessing.KEEP - val gcsOperations = - GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) - val bigQueryGcsStorageOperations = - BigQueryGcsStorageOperation( - gcsOperations, - gcsConfig, - gcsNameTransformer, - keepStagingFiles, - bigquery, - sqlGenerator, - destinationHandler, - ) - DefaultSyncOperation( - parsedCatalog, - destinationHandler, - config.datasetId, - { - initialStatus: DestinationInitialStatus, - disableTD -> - StagingStreamOperations( - bigQueryGcsStorageOperations, - initialStatus, - FileUploadFormat.CSV, - V2_WITH_GENERATION, - disableTD, - ) - }, - migrations, - config.disableTypingDeduping, - ) - } - } - } - - override fun createStreamLoader(stream: DestinationStream): StreamLoader { - return BigqueryStreamLoader(stream, bigquery, config) - } +) { + @Singleton + fun make() = TypingDedupingWriter( + catalog, + BigqueryInitialStateGatherer(), + BigqueryRawTableOperations(), + TypingDedupingFinalTableOperations(sqlGenerator, destinationHandler), + ) } // TODO delete this - this is definitely duplicated code, and also is definitely wrong // e.g. we need to handle special chars in stream name/namespace (c.f. -// bigquerysqlgenerator.buildStreamId) +// bigquerysqlgenerator.buildStreamId) // and that logic needs to be in BigqueryWriter.setup, to handle collisions // (probably actually a toolkit) object TempUtils { From 80e1447d277f68bd069519aa5054e7996d1e67e1 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 16:18:59 -0700 Subject: [PATCH 05/55] progress --- .../DestinationInitialStatusGatherer.kt | 16 +++- .../load/orchestration/DestinationNames.kt | 40 ++++++++++ .../LegacyTypingDedupingStreamLoader.kt | 6 +- .../TypingDedupingDestinationInitialStatus.kt | 12 +-- .../TypingDedupingRawTableOperations.kt | 8 +- .../TypingDedupingWriter.kt | 79 +++++++++++++++---- .../write/BigqueryInitialStateGatherer.kt | 11 ++- .../bigquery/write/BigqueryWriter.kt | 36 ++------- 8 files changed, 150 insertions(+), 58 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt index 3dd9f0bf16d00..b97ad211673f9 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt @@ -8,8 +8,22 @@ import io.airbyte.cdk.load.command.DestinationStream interface DestinationInitialStatus +/** + * Some destinations can efficiently fetch multiple tables' information in a single query, so this + * interface accepts multiple streams in a single method call. + * + * For destinations which do not support that optimization, a simpler implementation would be + * something like this: + * ```kotlin + * streams.forEach { (stream, (tableNames, columnNames)) -> + * launch { + * // ... gather state... + * } + * } + * ``` + */ fun interface DestinationInitialStatusGatherer { - fun gatherInitialStatus( + suspend fun gatherInitialStatus( streams: Map>, ): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index 9af3a0f8c0003..8801e609ddf48 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -1,5 +1,11 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration +import io.airbyte.cdk.load.command.DestinationStream + data class TableNames( // this is pretty dumb, but in theory we could have: // * old-style implementation: raw+final tables both exist @@ -13,6 +19,11 @@ data class TableNames( "At least one table name should be nonnull" } } + + fun conflictsWith(other: TableNames) = + this.rawTableName.conflictsWith(other.rawTableName) || + this.finalTableName.conflictsWith(other.finalTableName) + companion object { // TODO comment explaining this const val TMP_TABLE_SUFFIX = "_airbyte_tmp" @@ -21,6 +32,13 @@ data class TableNames( data class TableName(val namespace: String, val name: String) +fun TableName?.conflictsWith(other: TableName?): Boolean { + if (this == null || other == null) { + return false + } + return this.namespace == other.namespace && this.name == other.name +} + /** * map from the column name as declared in the schema, to the column name that we'll create in the * final (typed) table. @@ -28,3 +46,25 @@ data class TableName(val namespace: String, val name: String) @JvmInline value class ColumnNameMapping(private val columnNameMapping: Map) : Map by columnNameMapping + +fun interface TableNameGenerator { + fun getTableName(streamDescriptor: DestinationStream.Descriptor): TableName +} + +fun interface ColumnNameGenerator { + /** + * In some database/warehouses, there's a difference between how a name is _displayed_, and how + * the underlying engine actually treats it. For example, a column might be displayed as + * `CamelCaseColumn`, but the engine actually treats it as lowercase `camelcasecolumn`, or + * truncate it to `CamelCas`. + * + * This is relevant for handling collisions between column names. We need to know what name will + * be displayed to the user, since that's what we'll use in queries - but we also need to know + * the "canonical" name to check whether two columns will collide. + * + * (edgao: I actually can't think of an example offhand. This logic predates me, and possibly + * doesn't need to exist.) + */ + data class ColumnName(val displayName: String, val canonicalName: String) + fun getColumnName(column: String): ColumnName +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt index 1387214c00e07..5184e38d9502d 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream @@ -30,8 +34,6 @@ class LegacyTypingDedupingStreamLoader( } override suspend fun close(streamFailure: StreamProcessingFailed?) { - // TODO is this handled in the bulk load pipeline already? - rawTableOperations.cleanupStage(rawTableName) if (streamFailure == null) { // TODO only do this in truncate mode, do all the correct truncate stuff rawTableOperations.overwriteRawTable(rawTableName, suffix = "_airbyte_tmp") diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt index 98b4d8085cdc5..136f9a55205ba 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -1,12 +1,14 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.orchestration.DestinationInitialStatus import java.time.Instant data class TypingDedupingDestinationInitialStatus( - /** - * Initial status of the final table, or null if the table doesn't exist yet. - */ + /** Initial status of the final table, or null if the table doesn't exist yet. */ val finalTableStatus: FinalTableInitialStatus?, val rawTableStatus: RawTableInitialStatus?, val tempRawTableStatus: RawTableInitialStatus?, @@ -15,9 +17,7 @@ data class TypingDedupingDestinationInitialStatus( data class FinalTableInitialStatus( val isSchemaMismatch: Boolean, val isEmpty: Boolean, - /** - * The generation ID of _any_ record from the final table, or `null` if the table is empty. - */ + /** The generation ID of _any_ record from the final table, or `null` if the table is empty. */ val finalTableGenerationId: Long?, ) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt index 28f3e18571583..666e8e0dba4b4 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.orchestration.TableName @@ -35,8 +39,4 @@ interface TypingDedupingRawTableOperations { * @return The generation ID of a record in the raw table, or `null` if the raw table is empty. */ fun getRawTableGeneration(rawTableName: TableName, suffix: String): Long? - - // TODO this probably could be named better - /** Delete all staged data from blob storage. */ - fun cleanupStage(rawTableName: TableName) } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index cf934c1d3d0ba..ec340c5c5dcb1 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -6,15 +6,26 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.data.ObjectType +import io.airbyte.cdk.load.orchestration.ColumnNameGenerator import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.TableNameGenerator import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader +import java.util.concurrent.Executors +import kotlinx.coroutines.asCoroutineDispatcher +import kotlinx.coroutines.launch +import kotlinx.coroutines.runBlocking class TypingDedupingWriter( private val catalog: DestinationCatalog, - private val stateGatherer: DestinationInitialStatusGatherer, + private val rawTableTableNameGenerator: TableNameGenerator, + private val finalTableTableNameGenerator: TableNameGenerator, + private val finalTableColumnNameGenerator: ColumnNameGenerator, + private val stateGatherer: + DestinationInitialStatusGatherer, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, ) : DestinationWriter { @@ -23,19 +34,59 @@ class TypingDedupingWriter( Map override suspend fun setup() { - // TODO - // -1. figure out table/column names - // 0. gather state - // 1. execute migrations - // 2. soft reset - // 3. gather state - names = TODO() - val initialInitialStatuses: Map = - stateGatherer.gatherInitialStatus(names) - // TODO migrations - // TODO soft reset if needed - // TODO only refetch streams that need to be refetched - initialStatuses = stateGatherer.gatherInitialStatus(names) + // TODO handle collisions in table names + names = + catalog.streams.associateWith { stream -> + Pair( + TableNames( + rawTableName = rawTableTableNameGenerator.getTableName(stream.descriptor), + finalTableName = + finalTableTableNameGenerator.getTableName(stream.descriptor), + ), + ColumnNameMapping( + // TODO handle collisions in column names + (stream.schema as ObjectType).properties.mapValues { (columnName, _) -> + finalTableColumnNameGenerator.getColumnName(columnName).displayName + } + ) + ) + } + + Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> + val initialInitialStatuses: + Map = + stateGatherer.gatherInitialStatus(names) + + // TODO migrations + + // If we have a schema mismatch, then execute a soft reset. + val streamsNeedingSoftReset = + initialInitialStatuses.filter { (_, status) -> + // if the table doesn't exist, then by definition we don't have a schema + // mismatch. + status.finalTableStatus?.isSchemaMismatch ?: false + } + runBlocking(dispatcher) { + streamsNeedingSoftReset.forEach { (stream, _) -> + launch { + finalTableOperations.softResetFinalTable( + stream, + names[stream]!!.first.finalTableName!! + ) + } + } + } + + // Soft reset will modify the initial status of a table. + // Refetch their statuses. + val statusesAfterSoftReset = + stateGatherer.gatherInitialStatus( + names.filterKeys { streamsNeedingSoftReset.containsKey(it) } + ) + // TODO check whether this is true + // second map "wins" when adding two maps together, so we'll retain the newer statuses. + initialStatuses = initialInitialStatuses + statusesAfterSoftReset + } } override fun createStreamLoader(stream: DestinationStream): StreamLoader { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt index 4e79b18b1ac99..ae7859c42fec8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.destination.bigquery.write import io.airbyte.cdk.load.command.DestinationStream @@ -6,8 +10,11 @@ import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus -class BigqueryInitialStateGatherer : DestinationInitialStatusGatherer { - override fun gatherInitialStatus(streams: Map>): Map { +class BigqueryInitialStateGatherer : + DestinationInitialStatusGatherer { + override fun gatherInitialStatus( + streams: Map> + ): Map { TODO("Not yet implemented") } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index 4fb0fe79ad9e6..bb971abe88589 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -6,35 +6,12 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.TableId -import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.V2_WITH_GENERATION -import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig -import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer -import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations -import io.airbyte.cdk.integrations.destination.gcs.credential.GcsHmacKeyCredentialConfig -import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat -import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingFinalTableOperations import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingWriter -import io.airbyte.cdk.load.write.DestinationWriter -import io.airbyte.cdk.load.write.StreamLoader -import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation -import io.airbyte.integrations.base.destination.operation.StandardStreamOperation -import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus -import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog import io.airbyte.integrations.base.destination.typing_deduping.StreamId -import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDV2Migration -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState -import io.airbyte.integrations.destination.bigquery.migrators.BigqueryAirbyteMetaAndGenerationIdMigration -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation -import io.airbyte.integrations.destination.bigquery.operation.BigQueryGcsStorageOperation -import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing -import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator import io.micronaut.context.annotation.Factory @@ -49,12 +26,13 @@ class BigqueryWriterFactory( private val destinationHandler: BigQueryDestinationHandler, ) { @Singleton - fun make() = TypingDedupingWriter( - catalog, - BigqueryInitialStateGatherer(), - BigqueryRawTableOperations(), - TypingDedupingFinalTableOperations(sqlGenerator, destinationHandler), - ) + fun make() = + TypingDedupingWriter( + catalog, + BigqueryInitialStateGatherer(), + BigqueryRawTableOperations(), + TypingDedupingFinalTableOperations(sqlGenerator, destinationHandler), + ) } // TODO delete this - this is definitely duplicated code, and also is definitely wrong From 626fa832b8a0e411860b8f59fc3b7f05d72cca4e Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 16:21:03 -0700 Subject: [PATCH 06/55] rename --- .../TypingDedupingRawTableOperations.kt | 4 ++-- ...gDedupingStreamLoader.kt => TypingDedupingStreamLoader.kt} | 2 +- .../legacy_typing_deduping/TypingDedupingWriter.kt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/{LegacyTypingDedupingStreamLoader.kt => TypingDedupingStreamLoader.kt} (97%) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt index 666e8e0dba4b4..a8886585ad12f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt @@ -33,8 +33,8 @@ interface TypingDedupingRawTableOperations { * Get the generation of a single record in the raw table. Not necessarily the min or max * generation, just _any_ record. * - * [LegacyTypingDedupingStreamLoader] is responsible for orchestrating the raw tables so that - * the temp raw table always contains exactly one generation. + * [TypingDedupingStreamLoader] is responsible for orchestrating the raw tables so that the temp + * raw table always contains exactly one generation. * * @return The generation ID of a record in the raw table, or `null` if the raw table is empty. */ diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt similarity index 97% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 5184e38d9502d..8caa821443f95 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/LegacyTypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -9,7 +9,7 @@ import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader -class LegacyTypingDedupingStreamLoader( +class TypingDedupingStreamLoader( override val stream: DestinationStream, private val initialStatus: TypingDedupingDestinationInitialStatus, private val rawTableName: TableName, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index ec340c5c5dcb1..2bb059f74ad3e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -91,7 +91,7 @@ class TypingDedupingWriter( override fun createStreamLoader(stream: DestinationStream): StreamLoader { val tableNames = names[stream]!!.first - return LegacyTypingDedupingStreamLoader( + return TypingDedupingStreamLoader( stream, initialStatuses[stream]!!, tableNames.rawTableName!!, From 7d718d00f6a25a5bcfcc417b44cf72f9513734ea Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 14 Apr 2025 17:04:13 -0700 Subject: [PATCH 07/55] rough direct-load sketch --- .../DirectLoadSqlGenerator.kt | 7 ++++ .../DirectLoadTableOperations.kt | 39 +++++++++++++++++++ .../DirectLoadTableStreamLoader.kt | 26 +++++++++++++ .../DirectLoadTableWriter.kt | 17 ++++++++ 4 files changed, 89 insertions(+) create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt new file mode 100644 index 0000000000000..5be1868b1d58f --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt @@ -0,0 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.direct_load_table + +interface DirectLoadSqlGenerator {} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt new file mode 100644 index 0000000000000..ae05c607f8c4a --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt @@ -0,0 +1,39 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.direct_load_table + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.DestinationHandler +import io.airbyte.cdk.load.orchestration.TableName + +class DirectLoadTableOperations( + private val sqlGenerator: DirectLoadSqlGenerator, + private val destinationHandler: DestinationHandler, +) { + fun createTable( + stream: DestinationStream, + finalTableName: TableName, + suffix: String, + replace: Boolean + ) { + destinationHandler.execute(TODO()) + } + + fun alterTable( + stream: DestinationStream, + finalTableName: TableName, + ) { + // TODO we should figure out some reasonable abstraction for diffing existing+expected + // table schema + TODO() + } + + fun overwriteFinalTable( + finalTableName: TableName, + suffix: String, + ) { + TODO() + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt new file mode 100644 index 0000000000000..e3104c1061141 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt @@ -0,0 +1,26 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.direct_load_table + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.state.StreamProcessingFailed +import io.airbyte.cdk.load.write.StreamLoader + +class DirectLoadTableStreamLoader( + override val stream: DestinationStream, + val tableOperations: DirectLoadTableOperations, +) : StreamLoader { + override suspend fun start() { + // TODO + // * create table if not exists + // * alter table if needed + // * truncate refresh setup + } + + override suspend fun close(streamFailure: StreamProcessingFailed?) { + // TODO + // * commit truncate refresh + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt new file mode 100644 index 0000000000000..6a266310148fa --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt @@ -0,0 +1,17 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.direct_load_table + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.write.DestinationWriter +import io.airbyte.cdk.load.write.StreamLoader + +class DirectLoadTableWriter( + private val tableOperations: DirectLoadTableOperations, +) : DestinationWriter { + override fun createStreamLoader(stream: DestinationStream): StreamLoader { + return DirectLoadTableStreamLoader(stream, tableOperations) + } +} From e399c3c28ffb915091c03da344852f8c37492657 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 08:40:48 -0700 Subject: [PATCH 08/55] implement T+D framework --- .../load/orchestration/DestinationNames.kt | 8 +- .../TypingDedupingFinalTableOperations.kt | 95 +++++++++++++++-- .../TypingDedupingSqlGenerator.kt | 100 +++++++++++++++++- .../TypingDedupingStreamLoader.kt | 4 +- .../TypingDedupingWriter.kt | 5 +- 5 files changed, 195 insertions(+), 17 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index 8801e609ddf48..0633b708772eb 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -24,13 +24,19 @@ data class TableNames( this.rawTableName.conflictsWith(other.rawTableName) || this.finalTableName.conflictsWith(other.finalTableName) + fun prettyPrint() = + "Raw table: ${rawTableName?.prettyPrint()}; Final table: ${finalTableName?.prettyPrint()}" + companion object { // TODO comment explaining this const val TMP_TABLE_SUFFIX = "_airbyte_tmp" + const val SOFT_RESET_SUFFIX = "_ab_soft_reset" } } -data class TableName(val namespace: String, val name: String) +data class TableName(val namespace: String, val name: String) { + fun prettyPrint() = "$namespace.$name" +} fun TableName?.conflictsWith(other: TableName?): Boolean { if (this == null || other == null) { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 1c86986204a5e..edc681961a0db 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -1,10 +1,19 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.DestinationHandler import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.TableNames.Companion.SOFT_RESET_SUFFIX +import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant +private val logger = KotlinLogging.logger {} + class TypingDedupingFinalTableOperations( private val sqlGenerator: TypingDedupingSqlGenerator, private val destinationHandler: DestinationHandler, @@ -12,18 +21,44 @@ class TypingDedupingFinalTableOperations( fun createFinalTable( stream: DestinationStream, finalTableName: TableName, - suffix: String, + finalTableSuffix: String, replace: Boolean ) { - destinationHandler.execute(TODO()) + logger.info { + "Creating final table for stream ${stream.descriptor} with name ${finalTableName.prettyPrint()}" + } + destinationHandler.execute( + sqlGenerator.createFinalTable( + stream, + finalTableName, + finalTableSuffix, + replace = replace + ) + ) } /** Reset the final table using a temp table or ALTER existing table's columns. */ fun softResetFinalTable( stream: DestinationStream, - finalTableName: TableName, + tableNames: TableNames, ) { - destinationHandler.execute(TODO()) + logger.info { + "Executing soft reset for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()}" + } + destinationHandler.execute(sqlGenerator.prepareTablesForSoftReset(stream, tableNames)) + typeAndDedupe( + stream, + tableNames, + maxProcessedTimestamp = null, + finalTableSuffix = SOFT_RESET_SUFFIX, + ) + destinationHandler.execute( + sqlGenerator.overwriteFinalTable( + stream, + tableNames.finalTableName!!, + finalTableSuffix = SOFT_RESET_SUFFIX + ) + ) } /** @@ -31,19 +66,61 @@ class TypingDedupingFinalTableOperations( * specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE ... SELECT *, DROP TABLE */ fun overwriteFinalTable( + stream: DestinationStream, finalTableName: TableName, - suffix: String, + finalTableSuffix: String, ) { - TODO() + logger.info { + "Overwriting final table for stream ${stream.descriptor} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" + } + destinationHandler.execute( + sqlGenerator.overwriteFinalTable( + stream, + finalTableName, + finalTableSuffix = finalTableSuffix + ) + ) } fun typeAndDedupe( stream: DestinationStream, - rawTableName: TableName, - finalTableName: TableName, + tableNames: TableNames, maxProcessedTimestamp: Instant?, finalTableSuffix: String ) { - TODO() + try { + logger.info { + "Attempting typing and deduping for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix" + } + val unsafeSql = + sqlGenerator.updateFinalTable( + stream, + tableNames, + finalTableSuffix = finalTableSuffix, + maxProcessedTimestamp = maxProcessedTimestamp, + useExpensiveSaferCasting = false, + ) + destinationHandler.execute(unsafeSql) + } catch (e: Exception) { + if (sqlGenerator.supportsExpensiveSaferCasting) { + logger.info(e) { + "Encountered Exception on unsafe SQL for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, re-attempting with error handling" + } + val saferSql = + sqlGenerator.updateFinalTable( + stream, + tableNames, + finalTableSuffix = finalTableSuffix, + maxProcessedTimestamp = maxProcessedTimestamp, + useExpensiveSaferCasting = true, + ) + destinationHandler.execute(saferSql) + } else { + logger.info(e) { + "Encountered Exception on unsafe SQL for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, not retrying" + } + throw e + } + } } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index c081318ffbd3c..5cb2f8f2feafe 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -1,3 +1,101 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.legacy_typing_deduping -interface TypingDedupingSqlGenerator {} +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.Sql +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.TableNames.Companion.SOFT_RESET_SUFFIX +import java.time.Instant + +interface TypingDedupingSqlGenerator { + /** + * Generate a SQL statement to create a fresh table to match the given stream. + * + * The generated SQL should throw an exception if the table already exists and `replace` is + * false. + * + * @param finalTableSuffix A suffix to add to the stream name. Useful for full refresh overwrite + * syncs, where we write the entire sync to a temp table. + * @param replace If true, will overwrite an existing table. If false, will throw an exception + * if the table already exists. If you're passing a non-empty prefix, you likely want to set + * this to true. + */ + fun createFinalTable( + stream: DestinationStream, + tableName: TableName, + finalTableSuffix: String, + replace: Boolean + ): Sql + + /** + * Whether [updateFinalTable] actually generates different SQL when `useExpensiveSaferCasting` + * is enabled. Some destinations don't have this distinction, and should override this field to + * `false`. + */ + val supportsExpensiveSaferCasting: Boolean + get() = true + + /** + * Generate a SQL statement to copy new data from the raw table into the final table. + * + * Responsible for: + * + * * Pulling new raw records from a table (i.e. records with null _airbyte_loaded_at) + * * Extracting the JSON fields and casting to the appropriate types + * * Handling errors in those casts + * * Merging those typed records into an existing table + * * Updating the raw records with SET _airbyte_loaded_at = now() + * + * Implementing classes are recommended to break this into smaller methods, which can be tested + * in isolation. However, this interface only requires a single mega-method. + * + * @param finalTableSuffix the suffix of the final table to write to. If empty string, writes to + * the final table directly. Useful for full refresh overwrite syncs, where we write the entire + * sync to a temp table and then swap it into the final table at the end. + * + * @param minRawTimestamp The latest _airbyte_extracted_at for which all raw records with that + * timestamp have already been typed+deduped. Implementations MAY use this value in a + * `_airbyte_extracted_at > minRawTimestamp` filter on the raw table to improve query + * performance. + * @param useExpensiveSaferCasting often the data coming from the source can be faithfully + * represented in the destination without issue, and using a "CAST" expression works fine, + * however sometimes we get badly typed data. In these cases we can use a more expensive query + * which handles casting exceptions. + */ + fun updateFinalTable( + stream: DestinationStream, + tableNames: TableNames, + finalTableSuffix: String, + maxProcessedTimestamp: Instant?, + useExpensiveSaferCasting: Boolean, + ): Sql + + /** + * Drop the previous final table, and rename the new final table to match the old final table. + * + * This method may assume that the stream is an OVERWRITE stream, and that the final suffix is + * non-empty. Callers are responsible for verifying those are true. + */ + fun overwriteFinalTable( + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String + ): Sql + + fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql +} + +/** Typically we need to create a soft reset temporary table and clear loaded at values */ +fun TypingDedupingSqlGenerator.prepareTablesForSoftReset( + stream: DestinationStream, + tableNames: TableNames +): Sql { + val createTempTable = + createFinalTable(stream, tableNames.finalTableName!!, SOFT_RESET_SUFFIX, replace = true) + val clearLoadedAt = clearLoadedAt(stream, tableNames.rawTableName!!) + return Sql.concat(createTempTable, clearLoadedAt) +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 8caa821443f95..b25014594123b 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -28,7 +28,7 @@ class TypingDedupingStreamLoader( finalTableOperations.createFinalTable( stream, finalTableName, - suffix = "", + finalTableSuffix = "", replace = false, ) } @@ -45,7 +45,7 @@ class TypingDedupingStreamLoader( finalTableSuffix = "", ) // TODO extract constant for suffix - finalTableOperations.overwriteFinalTable(finalTableName, suffix = "") + finalTableOperations.overwriteFinalTable(finalTableName, finalTableSuffix = "") } } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 2bb059f74ad3e..1a8992198d002 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -69,10 +69,7 @@ class TypingDedupingWriter( runBlocking(dispatcher) { streamsNeedingSoftReset.forEach { (stream, _) -> launch { - finalTableOperations.softResetFinalTable( - stream, - names[stream]!!.first.finalTableName!! - ) + finalTableOperations.softResetFinalTable(stream, names[stream]!!.first) } } } From c43d10b7d365ec80ea48b0ab3a0a2367277a4c4f Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 08:48:28 -0700 Subject: [PATCH 09/55] fix stuff --- .../TypingDedupingStreamLoader.kt | 20 ++++++++++--------- .../TypingDedupingWriter.kt | 7 +++---- 2 files changed, 14 insertions(+), 13 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index b25014594123b..14c34e1436748 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -5,15 +5,14 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader class TypingDedupingStreamLoader( override val stream: DestinationStream, private val initialStatus: TypingDedupingDestinationInitialStatus, - private val rawTableName: TableName, - private val finalTableName: TableName, + private val tableNames: TableNames, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, ) : StreamLoader { @@ -21,13 +20,13 @@ class TypingDedupingStreamLoader( override suspend fun start() { // TODO do all the truncate stuff rawTableOperations.prepareRawTable( - rawTableName, + tableNames.rawTableName!!, suffix = "", replace = false, ) finalTableOperations.createFinalTable( stream, - finalTableName, + tableNames.finalTableName!!, finalTableSuffix = "", replace = false, ) @@ -36,16 +35,19 @@ class TypingDedupingStreamLoader( override suspend fun close(streamFailure: StreamProcessingFailed?) { if (streamFailure == null) { // TODO only do this in truncate mode, do all the correct truncate stuff - rawTableOperations.overwriteRawTable(rawTableName, suffix = "_airbyte_tmp") + rawTableOperations.overwriteRawTable(tableNames.rawTableName!!, suffix = "_airbyte_tmp") finalTableOperations.typeAndDedupe( stream, - rawTableName, - finalTableName, + tableNames, maxProcessedTimestamp = TODO(), finalTableSuffix = "", ) // TODO extract constant for suffix - finalTableOperations.overwriteFinalTable(finalTableName, finalTableSuffix = "") + finalTableOperations.overwriteFinalTable( + stream, + tableNames.finalTableName!!, + finalTableSuffix = "" + ) } } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 1a8992198d002..c877ba10f5462 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -57,7 +57,8 @@ class TypingDedupingWriter( Map = stateGatherer.gatherInitialStatus(names) - // TODO migrations + // TODO migrations - we should probably actually drop all existing migrations as part of + // this project, but eventually we'll need some solution here // If we have a schema mismatch, then execute a soft reset. val streamsNeedingSoftReset = @@ -80,7 +81,6 @@ class TypingDedupingWriter( stateGatherer.gatherInitialStatus( names.filterKeys { streamsNeedingSoftReset.containsKey(it) } ) - // TODO check whether this is true // second map "wins" when adding two maps together, so we'll retain the newer statuses. initialStatuses = initialInitialStatuses + statusesAfterSoftReset } @@ -91,8 +91,7 @@ class TypingDedupingWriter( return TypingDedupingStreamLoader( stream, initialStatuses[stream]!!, - tableNames.rawTableName!!, - tableNames.finalTableName!!, + tableNames, rawTableOperations, finalTableOperations, ) From b529a83155368f48d7302046c9038c7348ff7d84 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 09:31:55 -0700 Subject: [PATCH 10/55] progress --- .../load/orchestration/DestinationNames.kt | 1 + .../TypingDedupingDestinationInitialStatus.kt | 18 +- .../TypingDedupingFinalTableOperations.kt | 12 +- .../TypingDedupingStreamLoader.kt | 372 +++++++++++++++++- .../TypingDedupingWriter.kt | 2 + 5 files changed, 379 insertions(+), 26 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index 0633b708772eb..99c21d4f2730a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -28,6 +28,7 @@ data class TableNames( "Raw table: ${rawTableName?.prettyPrint()}; Final table: ${finalTableName?.prettyPrint()}" companion object { + const val NO_SUFFIX = "" // TODO comment explaining this const val TMP_TABLE_SUFFIX = "_airbyte_tmp" const val SOFT_RESET_SUFFIX = "_ab_soft_reset" diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt index 136f9a55205ba..3fd5cd7fc8df6 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -35,4 +35,20 @@ data class RawTableInitialStatus( * (note the strictly-greater comparison). */ val maxProcessedTimestamp: Instant?, -) +) { + companion object { + /** + * If the raw table doesn't exist, we'll obviously need to create it. After creating a raw + * table, this is its default state (i.e. it has no records, so there are by definition no + * unprocessed records, and no processed records). + */ + val emptyTableStatus = RawTableInitialStatus(false, maxProcessedTimestamp = null) + } +} + +/** + * Many callers need to do a `create table if not exists`. This is a utility method to update the + * initial status accordingly - i.e. if the table already existed, retain its status; otherwise, use + * the empty table status. + */ +fun RawTableInitialStatus?.reify() = this ?: RawTableInitialStatus.emptyTableStatus diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index edc681961a0db..ec8b523cebfb7 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -25,7 +25,7 @@ class TypingDedupingFinalTableOperations( replace: Boolean ) { logger.info { - "Creating final table for stream ${stream.descriptor} with name ${finalTableName.prettyPrint()}" + "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()}" } destinationHandler.execute( sqlGenerator.createFinalTable( @@ -43,7 +43,7 @@ class TypingDedupingFinalTableOperations( tableNames: TableNames, ) { logger.info { - "Executing soft reset for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()}" + "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" } destinationHandler.execute(sqlGenerator.prepareTablesForSoftReset(stream, tableNames)) typeAndDedupe( @@ -71,7 +71,7 @@ class TypingDedupingFinalTableOperations( finalTableSuffix: String, ) { logger.info { - "Overwriting final table for stream ${stream.descriptor} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" + "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" } destinationHandler.execute( sqlGenerator.overwriteFinalTable( @@ -90,7 +90,7 @@ class TypingDedupingFinalTableOperations( ) { try { logger.info { - "Attempting typing and deduping for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix" + "Attempting typing and deduping for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix" } val unsafeSql = sqlGenerator.updateFinalTable( @@ -104,7 +104,7 @@ class TypingDedupingFinalTableOperations( } catch (e: Exception) { if (sqlGenerator.supportsExpensiveSaferCasting) { logger.info(e) { - "Encountered Exception on unsafe SQL for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, re-attempting with error handling" + "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, re-attempting with error handling" } val saferSql = sqlGenerator.updateFinalTable( @@ -117,7 +117,7 @@ class TypingDedupingFinalTableOperations( destinationHandler.execute(saferSql) } else { logger.info(e) { - "Encountered Exception on unsafe SQL for stream ${stream.descriptor} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, not retrying" + "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, not retrying" } throw e } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 14c34e1436748..b530103187cf2 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -6,8 +6,14 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.TableNames.Companion.NO_SUFFIX +import io.airbyte.cdk.load.orchestration.TableNames.Companion.TMP_TABLE_SUFFIX import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader +import io.github.oshai.kotlinlogging.KotlinLogging +import java.time.Instant + +private val logger = KotlinLogging.logger {} class TypingDedupingStreamLoader( override val stream: DestinationStream, @@ -15,39 +21,367 @@ class TypingDedupingStreamLoader( private val tableNames: TableNames, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : StreamLoader { + private val isTruncateSync = + when (stream.minimumGenerationId) { + 0L -> false + stream.generationId -> true + else -> { + throw IllegalArgumentException("Hybrid refreshes are not yet supported.") + } + } + private lateinit var rawTableSuffix: String + private lateinit var finalTmpTableSuffix: String + /** + * The status of the raw table that "matters" for this sync. Specifically: + * * For normal syncs / merge refreshes, this is the status of the real raw table) + * * For truncate refreshes, this is the status of the temp raw table (because we never even + * look at the real raw table) + */ + private lateinit var initialRawTableStatus: RawTableInitialStatus override suspend fun start() { - // TODO do all the truncate stuff - rawTableOperations.prepareRawTable( - tableNames.rawTableName!!, - suffix = "", - replace = false, - ) - finalTableOperations.createFinalTable( - stream, - tableNames.finalTableName!!, - finalTableSuffix = "", - replace = false, - ) + if (isTruncateSync) { + val (rawTableStatus, suffix) = prepareStageForTruncate() + initialRawTableStatus = rawTableStatus + rawTableSuffix = suffix + } else { + rawTableSuffix = NO_SUFFIX + initialRawTableStatus = prepareStageForNormalSync() + } + + if (!disableTypeDedupe) { + // Prepare final tables based on sync mode. + finalTmpTableSuffix = prepareFinalTable() + } else { + logger.info { "Typing and deduping disabled, skipping final table initialization" } + finalTmpTableSuffix = NO_SUFFIX + } + } + + private fun prepareStageForTruncate(): Pair { + /* + tl;dr: + * if a temp raw table exists, check whether it belongs to the correct generation. + * if wrong generation, truncate it. + * regardless, write into the temp raw table. + * else, if a real raw table exists, check its generation. + * if wrong generation, write into a new temp raw table. + * else, write into the preexisting real raw table. + * else, create a new temp raw table and write into it. + */ + if (initialStatus.tempRawTableStatus != null) { + val tempStageGeneration = + rawTableOperations.getRawTableGeneration( + tableNames.rawTableName!!, + TMP_TABLE_SUFFIX + ) + if (tempStageGeneration == null || tempStageGeneration == stream.generationId) { + logger.info { + "${stream.descriptor.toPrettyString()}: truncate sync, and existing temp raw table belongs to generation $tempStageGeneration (== current generation ${stream.generationId}). Retaining it." + } + // The temp table is from the correct generation. Set up any other resources + // (staging file, etc.), but leave the table untouched. + rawTableOperations.prepareRawTable( + tableNames.rawTableName, + TMP_TABLE_SUFFIX, + ) + return Pair(initialStatus.tempRawTableStatus.reify(), TMP_TABLE_SUFFIX) + } else { + logger.info { + "${stream.descriptor.toPrettyString()}: truncate sync, and existing temp raw table belongs to generation $tempStageGeneration (!= current generation ${stream.generationId}). Truncating it." + } + // The temp stage is from the wrong generation. Nuke it. + rawTableOperations.prepareRawTable( + tableNames.rawTableName, + TMP_TABLE_SUFFIX, + replace = true, + ) + // We nuked the temp raw table, so create a new initial raw table status. + return Pair( + RawTableInitialStatus.emptyTableStatus, + TMP_TABLE_SUFFIX, + ) + } + } else if (initialStatus.rawTableStatus != null) { + // It's possible to "resume" a truncate sync that was previously already finalized. + // In this case, there is no existing temp raw table, and there is a real raw table + // which already belongs to the correct generation. + // Check for that case now. + val realStageGeneration = + rawTableOperations.getRawTableGeneration(tableNames.rawTableName!!, NO_SUFFIX) + if (realStageGeneration == null || realStageGeneration == stream.generationId) { + logger.info { + "${stream.descriptor.toPrettyString()}: truncate sync, no existing temp raw table, and existing real raw table belongs to generation $realStageGeneration (== current generation ${stream.generationId}). Retaining it." + } + // The real raw table is from the correct generation. Set up any other resources + // (staging file, etc.), but leave the table untouched. + rawTableOperations.prepareRawTable(tableNames.rawTableName, NO_SUFFIX) + return Pair(initialStatus.rawTableStatus.reify(), NO_SUFFIX) + } else { + logger.info { + "${stream.descriptor.toPrettyString()}: truncate sync, existing real raw table belongs to generation $realStageGeneration (!= current generation ${stream.generationId}), and no preexisting temp raw table. Creating a temp raw table." + } + // We're initiating a new truncate refresh. Create a new temp stage. + rawTableOperations.prepareRawTable( + tableNames.rawTableName, + TMP_TABLE_SUFFIX, + ) + return Pair( + // Create a fresh raw table status, since we created a fresh temp stage. + RawTableInitialStatus.emptyTableStatus, + TMP_TABLE_SUFFIX, + ) + } + } else { + logger.info { + "${stream.descriptor.toPrettyString()}: truncate sync, and no preexisting temp or raw table. Creating a temp raw table." + } + // We're initiating a new truncate refresh. Create a new temp stage. + rawTableOperations.prepareRawTable( + tableNames.rawTableName!!, + TMP_TABLE_SUFFIX, + ) + return Pair( + // Create a fresh raw table status, since we created a fresh temp stage. + RawTableInitialStatus.emptyTableStatus, + TMP_TABLE_SUFFIX, + ) + } + } + + private fun prepareStageForNormalSync(): RawTableInitialStatus { + logger.info { + "${stream.descriptor.toPrettyString()}: non-truncate sync. Creating raw table if not exists." + } + rawTableOperations.prepareRawTable(tableNames.rawTableName!!, NO_SUFFIX) + if (initialStatus.tempRawTableStatus != null) { + logger.info { + "${stream.descriptor.toPrettyString()}: non-truncate sync, but temp raw table exists. Transferring it to real raw table." + } + // There was a previous truncate refresh attempt, which failed, and left some + // records behind. + // Retrieve those records and put them in the real stage. + // This is necessary to avoid certain data loss scenarios. + // (specifically: a user initiates a truncate sync, which fails, but emits some records. + // It also emits a state message for "resumable" full refresh. + // The user then initiates an incremental sync, which runs using that state. + // In this case, we MUST retain the records from the truncate attempt.) + rawTableOperations.transferFromTempRawTable(tableNames.rawTableName, TMP_TABLE_SUFFIX) + + // We need to combine the raw table statuses from the real and temp raw tables. + val hasUnprocessedRecords = + initialStatus.tempRawTableStatus.hasUnprocessedRecords || + (initialStatus.rawTableStatus?.hasUnprocessedRecords ?: false) + // Pick the earlier min timestamp. + val maxProcessedTimestamp: Instant? = + initialStatus.rawTableStatus?.maxProcessedTimestamp?.let { realRawTableTimestamp -> + initialStatus.tempRawTableStatus.maxProcessedTimestamp?.let { + tempRawTableTimestamp -> + if (realRawTableTimestamp.isBefore(tempRawTableTimestamp)) { + realRawTableTimestamp + } else { + tempRawTableTimestamp + } + } + ?: realRawTableTimestamp + } + ?: initialStatus.tempRawTableStatus.maxProcessedTimestamp + val updatedStatus = + RawTableInitialStatus( + hasUnprocessedRecords = hasUnprocessedRecords, + maxProcessedTimestamp = maxProcessedTimestamp, + ) + logger.info { + "${stream.descriptor.toPrettyString()}: After record transfer, initial raw table status is $updatedStatus." + } + return updatedStatus + } else { + logger.info { + "${stream.descriptor.toPrettyString()}: non-truncate sync and no temp raw table. Initial raw table status is $initialRawTableStatus." + } + return initialStatus.rawTableStatus.reify() + } + } + + private fun prepareFinalTable(): String { + // No special handling if final table doesn't exist, just create and return + if (initialStatus.finalTableStatus == null) { + logger.info { + "Final table does not exist for stream ${stream.descriptor.toPrettyString()}, creating ${tableNames.finalTableName!!.prettyPrint()}." + } + finalTableOperations.createFinalTable( + stream, + tableNames.finalTableName!!, + NO_SUFFIX, + replace = false + ) + return NO_SUFFIX + } + + logger.info { "Final Table exists for stream ${stream.descriptor.toPrettyString()}" } + // The table already exists. Decide whether we're writing to it directly, or + // using a tmp table. + if (isTruncateSync) { + if ( + initialStatus.finalTableStatus.isEmpty || + initialStatus.finalTableStatus.finalTableGenerationId == null + ) { + if (!initialStatus.finalTableStatus.isSchemaMismatch) { + logger.info { + "Truncate sync, and final table is empty and has correct schema. Writing to it directly." + } + return NO_SUFFIX + } else { + // No point soft resetting an empty table. We'll just do an overwrite later. + logger.info { + "Truncate sync, and final table is empty, but has the wrong schema. Using a temp final table." + } + return prepareFinalTableForOverwrite() + } + } else if ( + initialStatus.finalTableStatus.finalTableGenerationId >= stream.minimumGenerationId + ) { + if (!initialStatus.finalTableStatus.isSchemaMismatch) { + logger.info { + "Truncate sync, and final table matches our generation and has correct schema. Writing to it directly." + } + return NO_SUFFIX + } else { + logger.info { + "Truncate sync, and final table matches our generation, but has the wrong schema. Writing to it directly, but triggering a soft reset first." + } + finalTableOperations.softResetFinalTable(stream, tableNames) + return NO_SUFFIX + } + } else { + // The final table is in the wrong generation. Use a temp final table. + return prepareFinalTableForOverwrite() + } + } else { + if (initialStatus.finalTableStatus.isSchemaMismatch) { + // We're loading data directly into the existing table. + // Make sure it has the right schema. + // Also, if a raw table migration wants us to do a soft reset, do that + // here. + logger.info { "Executing soft-reset on final table of stream ${stream.descriptor}" } + finalTableOperations.softResetFinalTable(stream, tableNames) + } + return NO_SUFFIX + } + } + + private fun prepareFinalTableForOverwrite(): String { + if ( + initialStatus.finalTableStatus?.isEmpty != true || + initialStatus.finalTableStatus.isSchemaMismatch + ) { + // overwrite an existing tmp table if needed. + finalTableOperations.createFinalTable( + stream, + tableNames.finalTableName!!, + TMP_TABLE_SUFFIX, + replace = true + ) + logger.info { + "Using temp final table for table ${stream.descriptor.toPrettyString()}, this will be overwritten at end of sync" + } + // We want to overwrite an existing table. Write into a tmp table. + // We'll overwrite the table at the end of the sync. + return TMP_TABLE_SUFFIX + } + + logger.info { + "Final Table for stream ${stream.descriptor.toPrettyString()} is empty and matches the expected v2 format, writing to table directly" + } + return NO_SUFFIX } override suspend fun close(streamFailure: StreamProcessingFailed?) { - if (streamFailure == null) { - // TODO only do this in truncate mode, do all the correct truncate stuff - rawTableOperations.overwriteRawTable(tableNames.rawTableName!!, suffix = "_airbyte_tmp") + val streamSuccessful = streamFailure == null + // Overwrite the raw table before doing anything else. + // This ensures that if T+D fails, we can easily retain the records on the next sync. + // It also means we don't need to run T+D using the temp raw table, + // which is possible (`typeAndDedupe(streamConfig.id.copy(rawName = streamConfig.id.rawName + // + suffix))` + // but annoying and confusing. + if (isTruncateSync && streamSuccessful && rawTableSuffix.isNotEmpty()) { + logger.info { + "Overwriting raw table for ${stream.descriptor.toPrettyString()} because this is a truncate sync, we received a stream success message, and are using a temporary raw table." + } + rawTableOperations.overwriteRawTable(tableNames.rawTableName!!, rawTableSuffix) + } else { + logger.info { + "Not overwriting raw table for ${stream.descriptor.toPrettyString()}. Truncate sync: $isTruncateSync; stream success: $streamSuccessful; raw table suffix: \"$rawTableSuffix\"" + } + } + + if (disableTypeDedupe) { + logger.info { + "Typing and deduping disabled, skipping final table finalization. Raw records can be found at ${tableNames.rawTableName!!.prettyPrint()}" + } + return + } + + // Normal syncs should T+D regardless of status, so the user sees progress after every + // attempt. + // We know this is a normal sync, so initialRawTableStatus is nonnull. + if ( + !isTruncateSync && + syncSummary.recordsWritten == 0L && + !initialRawTableStatus.hasUnprocessedRecords + ) { + logger.info { + "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} because it had no records during this sync and no unprocessed records from a previous sync." + } + } else if ( + isTruncateSync && + (!streamSuccessful || + (syncSummary.recordsWritten == 0L && + !initialRawTableStatus.hasUnprocessedRecords)) + ) { + // But truncate syncs should only T+D if the sync was successful, since we're T+Ding + // into a temp final table anyway. + // We only run T+D if the current sync had some records, or a previous attempt wrote + // some records to the temp raw table. + logger.info { + "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} running as truncate sync. Stream success: $streamSuccessful; records written: ${syncSummary.recordsWritten}; temp raw table already existed: ${initialRawTableStatus.rawTableExists}; temp raw table had records: ${initialRawTableStatus.hasUnprocessedRecords}" + } + } else { + // When targeting the temp final table, we want to read all the raw records + // because the temp final table is always a full rebuild. Typically, this is equivalent + // to filtering on timestamp, but might as well be explicit. + val maxProcessedTimestamp = + if (finalTmpTableSuffix.isEmpty()) { + initialRawTableStatus.maxProcessedTimestamp + } else { + null + } finalTableOperations.typeAndDedupe( stream, tableNames, - maxProcessedTimestamp = TODO(), - finalTableSuffix = "", + maxProcessedTimestamp = maxProcessedTimestamp, + finalTableSuffix = finalTmpTableSuffix ) - // TODO extract constant for suffix + } + + // We want to run this independently of whether we ran T+D. + // E.g. it's valid for a sync to emit 0 records (e.g. the source table is legitimately + // empty), in which case we want to overwrite the final table with an empty table. + if (isTruncateSync && streamSuccessful && finalTmpTableSuffix.isNotBlank()) { + logger.info { + "Overwriting final table for ${stream.descriptor.toPrettyString()} because this is a truncate sync, we received a stream success message, and we are using a temp final table.." + } finalTableOperations.overwriteFinalTable( stream, tableNames.finalTableName!!, - finalTableSuffix = "" + finalTableSuffix = finalTmpTableSuffix ) + } else { + logger.info { + "Not overwriting final table for ${stream.descriptor.toPrettyString()}. Truncate sync: $isTruncateSync; stream success: $streamSuccessful; final table suffix not blank: ${finalTmpTableSuffix.isNotBlank()}" + } } } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index c877ba10f5462..142f8d38f80b3 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -28,6 +28,7 @@ class TypingDedupingWriter( DestinationInitialStatusGatherer, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : DestinationWriter { private lateinit var names: Map> private lateinit var initialStatuses: @@ -94,6 +95,7 @@ class TypingDedupingWriter( tableNames, rawTableOperations, finalTableOperations, + disableTypeDedupe = disableTypeDedupe, ) } } From 045786505ff30cd5d9337d6cf37383ffc898397e Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 09:52:06 -0700 Subject: [PATCH 11/55] derp --- .../legacy_typing_deduping/TypingDedupingStreamLoader.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index b530103187cf2..74f17f19c3546 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -346,7 +346,7 @@ class TypingDedupingStreamLoader( // We only run T+D if the current sync had some records, or a previous attempt wrote // some records to the temp raw table. logger.info { - "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} running as truncate sync. Stream success: $streamSuccessful; records written: ${syncSummary.recordsWritten}; temp raw table already existed: ${initialRawTableStatus.rawTableExists}; temp raw table had records: ${initialRawTableStatus.hasUnprocessedRecords}" + "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} running as truncate sync. Stream success: $streamSuccessful; records written: ${syncSummary.recordsWritten}; temp raw table had records: ${initialRawTableStatus.hasUnprocessedRecords}" } } else { // When targeting the temp final table, we want to read all the raw records From 063c79def48dad1417999e103dcee59a787f746f Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 11:13:13 -0700 Subject: [PATCH 12/55] start porting to new iface --- .../BigQueryDestinationHandler.kt | 372 ++---------------- .../typing_deduping/BigQuerySqlGenerator.kt | 18 +- ...igqueryDestinationInitialStatusGatherer.kt | 300 ++++++++++++++ 3 files changed, 348 insertions(+), 342 deletions(-) create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt index d10bb0459f176..178931dc4f7d6 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt @@ -3,122 +3,31 @@ */ package io.airbyte.integrations.destination.bigquery.typing_deduping -import com.google.cloud.bigquery.* -import com.google.common.annotations.VisibleForTesting +import com.google.cloud.bigquery.BigQuery +import com.google.cloud.bigquery.BigQueryException +import com.google.cloud.bigquery.Job +import com.google.cloud.bigquery.JobConfiguration +import com.google.cloud.bigquery.JobId +import com.google.cloud.bigquery.JobInfo +import com.google.cloud.bigquery.JobStatistics +import com.google.cloud.bigquery.JobStatus +import com.google.cloud.bigquery.QueryJobConfiguration import com.google.common.collect.Streams import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler -import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.util.ConnectorExceptionUtil +import io.airbyte.cdk.load.orchestration.DestinationHandler +import io.airbyte.cdk.load.orchestration.Sql import io.airbyte.commons.exceptions.ConfigErrorException -import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation -import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation.Companion.TMP_TABLE_SUFFIX -import io.airbyte.integrations.base.destination.typing_deduping.* -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsIgnoreCase -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.matchingKey import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState -import java.math.BigInteger -import java.util.* -import java.util.function.Consumer -import java.util.function.Function -import java.util.function.Predicate -import java.util.stream.Collectors +import io.github.oshai.kotlinlogging.KotlinLogging +import java.util.UUID import java.util.stream.Stream import kotlin.math.min -import org.apache.commons.text.StringSubstitutor -import org.slf4j.Logger -import org.slf4j.LoggerFactory -class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLocation: String) : - DestinationHandler { - fun findExistingTable(id: StreamId): Optional { - val table = bq.getTable(id.finalNamespace, id.finalName) - return Optional.ofNullable(table).map { obj: Table -> obj.getDefinition() } - } - - fun isFinalTableEmpty(id: StreamId): Boolean { - return BigInteger.ZERO == bq.getTable(TableId.of(id.finalNamespace, id.finalName)).numRows - } - - @Throws(Exception::class) - fun getInitialRawTableState(id: StreamId, suffix: String): InitialRawTableStatus { - bq.getTable(TableId.of(id.rawNamespace, id.rawName + suffix)) - ?: // Table doesn't exist. There are no unprocessed records, and no timestamp. - return InitialRawTableStatus(false, false, Optional.empty()) - - val unloadedRecordTimestamp = - bq.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - java.util.Map.of( - "raw_table", - id.rawTableId(BigQuerySqlGenerator.Companion.QUOTE, suffix) - ) - ) - .replace( // bigquery timestamps have microsecond precision - """ - SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND) - FROM ${'$'}{raw_table} - WHERE _airbyte_loaded_at IS NULL - - """.trimIndent() - ) - ) - .build() - ) - .iterateAll() - .iterator() - .next() - .first() - // If this value is null, then there are no records with null loaded_at. - // If it's not null, then we can return immediately - we've found some unprocessed records - // and their - // timestamp. - if (!unloadedRecordTimestamp.isNull) { - return InitialRawTableStatus( - true, - true, - Optional.of(unloadedRecordTimestamp.timestampInstant) - ) - } - - val loadedRecordTimestamp = - bq.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - java.util.Map.of( - "raw_table", - id.rawTableId(BigQuerySqlGenerator.Companion.QUOTE, suffix) - ) - ) - .replace( - """ - SELECT MAX(_airbyte_extracted_at) - FROM ${'$'}{raw_table} - - """.trimIndent() - ) - ) - .build() - ) - .iterateAll() - .iterator() - .next() - .first() - // We know (from the previous query) that all records have been processed by T+D already. - // So we just need to get the timestamp of the most recent record. - return if (loadedRecordTimestamp.isNull) { - // Null timestamp because the table is empty. T+D can process the entire raw table - // during this sync. - InitialRawTableStatus(true, false, Optional.empty()) - } else { - // The raw table already has some records. T+D can skip all records with timestamp <= - // this value. - InitialRawTableStatus(true, false, Optional.of(loadedRecordTimestamp.timestampInstant)) - } - } +private val logger = KotlinLogging.logger {} +class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLocation: String) : + DestinationHandler { @Throws(InterruptedException::class) override fun execute(sql: Sql) { val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION") @@ -127,7 +36,7 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo } val queryId = UUID.randomUUID() val statement = java.lang.String.join("\n", transactions) - LOGGER.debug("Executing sql {}: {}", queryId, statement) + logger.debug { "Executing sql $queryId: $statement" } /* * If you run a query like CREATE SCHEMA ... OPTIONS(location=foo); CREATE TABLE ...;, bigquery @@ -154,13 +63,9 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo } val statistics = job.getStatistics() - LOGGER.debug( - "Root-level job {} completed in {} ms; processed {} bytes; billed for {} bytes", - queryId, - statistics.endTime - statistics.startTime, - statistics.totalBytesProcessed, - statistics.totalBytesBilled - ) + logger.debug { + "Root-level job $queryId completed in ${statistics.endTime - statistics.startTime} ms; processed ${statistics.totalBytesProcessed} bytes; billed for ${statistics.totalBytesBilled} bytes" + } // SQL transactions can spawn child jobs, which are billed individually. Log their stats // too. @@ -190,238 +95,33 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo if (truncatedQuery != configuration.query) { truncatedQuery += "..." } - LOGGER.debug( - "Child sql {} completed in {} ms; processed {} bytes; billed for {} bytes", - truncatedQuery, - childQueryStats.endTime - childQueryStats.startTime, - childQueryStats.totalBytesProcessed, - childQueryStats.totalBytesBilled - ) + logger.debug { + "Child sql $truncatedQuery completed in ${childQueryStats.endTime - childQueryStats.startTime} ms; processed ${childQueryStats.totalBytesProcessed} bytes; billed for ${childQueryStats.totalBytesBilled} bytes" + } } else { // other job types are extract/copy/load // we're probably not using them, but handle just in case? val childJobStats = childJob.getStatistics() - LOGGER.debug( - "Non-query child job ({}) completed in {} ms", - configuration.type, - childJobStats.endTime - childJobStats.startTime - ) + logger.debug { + "Non-query child job (${configuration.type}) completed in ${childJobStats.endTime - childJobStats.startTime} ms" + } } } } } - @Throws(Exception::class) - override fun gatherInitialState( - streamConfigs: List - ): List> { - val initialStates: MutableList> = - ArrayList() - for (streamConfig in streamConfigs) { - val id = streamConfig.id - val finalTable = findExistingTable(id) - val rawTableState = getInitialRawTableState(id, "") - val tempRawTableState = - getInitialRawTableState(id, AbstractStreamOperation.TMP_TABLE_SUFFIX) - initialStates.add( - DestinationInitialStatus( - streamConfig, - finalTable.isPresent, - rawTableState, - tempRawTableState, - finalTable.isPresent && - !existingSchemaMatchesStreamConfig(streamConfig, finalTable.get()), - finalTable.isEmpty || - isFinalTableEmpty( - id - ), // Return a default state blob since we don't actually track state. - BigQueryDestinationState(false), - // for now, just use 0. this means we will always use a temp final table. - // platform has a workaround for this, so it's OK. - // TODO only fetch this on truncate syncs - // TODO once we have destination state, use that instead of a query - finalTableGenerationId = 0, - // temp table is always empty until we commit, so always return null - finalTempTableGenerationId = null, - ) - ) - } - return initialStates - } - - @Throws(Exception::class) - override fun commitDestinationStates( - destinationStates: Map - ) { - // Intentionally do nothing. Bigquery doesn't actually support destination states. - } - - @Throws(TableNotMigratedException::class) - private fun existingSchemaMatchesStreamConfig( - stream: StreamConfig, - existingTable: TableDefinition - ): Boolean { - val alterTableReport = buildAlterTableReport(stream, existingTable) - var tableClusteringMatches = false - var tablePartitioningMatches = false - if (existingTable is StandardTableDefinition) { - tableClusteringMatches = clusteringMatches(stream, existingTable) - tablePartitioningMatches = partitioningMatches(existingTable) - } - LOGGER.info( - "Alter Table Report {} {} {}; Clustering {}; Partitioning {}", - alterTableReport.columnsToAdd, - alterTableReport.columnsToRemove, - alterTableReport.columnsToChangeType, - tableClusteringMatches, - tablePartitioningMatches - ) - - return alterTableReport.isNoOp && tableClusteringMatches && tablePartitioningMatches - } - - fun buildAlterTableReport( - stream: StreamConfig, - existingTable: TableDefinition - ): AlterTableReport { - val pks = getPks(stream) - - val streamSchema: Map = - stream.columns.entries.associate { - it.key.name to BigQuerySqlGenerator.toDialectType(it.value) - } - - val existingSchema = - existingTable.schema!! - .fields - .stream() - .collect( - Collectors.toMap( - Function { field: Field -> field.name }, - Function { field: Field -> field.type.standardType } - ) - ) - - // Columns in the StreamConfig that don't exist in the TableDefinition - val columnsToAdd = - streamSchema.keys - .stream() - .filter { name: String -> !containsIgnoreCase(existingSchema.keys, name) } - .collect(Collectors.toSet()) - - // Columns in the current schema that are no longer in the StreamConfig - val columnsToRemove = - existingSchema.keys - .stream() - .filter { name: String -> - !containsIgnoreCase(streamSchema.keys, name) && - !containsIgnoreCase(JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS, name) + override fun createNamespaces(namespaces: List) { + namespaces.forEach { dataset -> + logger.info { "Creating dataset if not present $dataset" } + try { + BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation) + } catch (e: BigQueryException) { + if (ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code)) { + throw ConfigErrorException(e.message!!, e) + } else { + throw e } - .collect(Collectors.toSet()) - - // Columns that are typed differently than the StreamConfig - val columnsToChangeType = - Stream.concat( - streamSchema.keys - .stream() // If it's not in the existing schema, it should already be in the - // columnsToAdd Set - .filter { name: String -> - matchingKey( - existingSchema.keys, - name - ) // if it does exist, only include it in this set if the type (the - // value in each respective map) - // is different between the stream and existing schemas - .map { key: String -> - existingSchema[key] != streamSchema[name] - } // if there is no matching key, then don't include it because it - // is probably already in columnsToAdd - .orElse(false) - }, // OR columns that used to have a non-null constraint and shouldn't - // (https://github.com/airbytehq/airbyte/pull/31082) - - existingTable.schema!! - .fields - .stream() - .filter { field: Field -> pks.contains(field.name) } - .filter { field: Field -> field.mode == Field.Mode.REQUIRED } - .map { obj: Field -> obj.name } - ) - .collect(Collectors.toSet()) - - val isDestinationV2Format = schemaContainAllFinalTableV2AirbyteColumns(existingSchema.keys) - - return AlterTableReport( - columnsToAdd, - columnsToRemove, - columnsToChangeType, - isDestinationV2Format - ) - } - - override fun createNamespaces(schemas: Set) { - schemas.forEach(Consumer { dataset: String -> this.createDataset(dataset) }) - } - - private fun createDataset(dataset: String) { - LOGGER.info("Creating dataset if not present {}", dataset) - try { - BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation) - } catch (e: BigQueryException) { - if (ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code)) { - throw ConfigErrorException(e.message!!, e) - } else { - throw e } } } - - companion object { - private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDestinationHandler::class.java) - - @JvmStatic - @VisibleForTesting - fun clusteringMatches( - stream: StreamConfig, - existingTable: StandardTableDefinition - ): Boolean { - return (existingTable.clustering != null && - containsAllIgnoreCase( - HashSet(existingTable.clustering!!.fields), - BigQuerySqlGenerator.Companion.clusteringColumns(stream) - )) - } - - @JvmStatic - @VisibleForTesting - fun partitioningMatches(existingTable: StandardTableDefinition): Boolean { - return existingTable.timePartitioning != null && - existingTable.timePartitioning!! - .field - .equals("_airbyte_extracted_at", ignoreCase = true) && - TimePartitioning.Type.DAY == existingTable.timePartitioning!!.type - } - - /** - * Checks the schema to determine whether the table contains all expected final table - * airbyte columns - * - * @param columnNames the column names of the schema to check - * @return whether all the [JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS] are present - */ - @VisibleForTesting - @JvmStatic - fun schemaContainAllFinalTableV2AirbyteColumns(columnNames: Collection?): Boolean { - return JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.stream() - .allMatch( - Predicate { column: String? -> - containsIgnoreCase(columnNames!!, column!!) - } - ) - } - - private fun getPks(stream: StreamConfig): Set { - return stream.primaryKey.map(ColumnId::name).toSet() - } - } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 7105ab26d681f..325e5c5bc792d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -5,6 +5,9 @@ package io.airbyte.integrations.destination.bigquery.typing_deduping import com.google.cloud.bigquery.StandardSQLTypeName import com.google.common.annotations.VisibleForTesting +import io.airbyte.cdk.load.command.Dedupe +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.integrations.base.destination.typing_deduping.* import io.airbyte.integrations.base.destination.typing_deduping.Array import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer @@ -635,8 +638,7 @@ class BigQuerySqlGenerator } // TODO maybe make this a BiMap and elevate this method and its inverse - // (toDestinationSQLType?) to - // the SQLGenerator? + // (toDestinationSQLType?) to the SQLGenerator? fun toDialectType(airbyteProtocolType: AirbyteProtocolType): StandardSQLTypeName { return when (airbyteProtocolType) { AirbyteProtocolType.STRING, @@ -652,15 +654,19 @@ class BigQuerySqlGenerator } } - fun clusteringColumns(stream: StreamConfig): List { + fun clusteringColumns( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping + ): List { val clusterColumns: MutableList = ArrayList() - if (stream.postImportAction == ImportType.DEDUPE) { + if (stream.importType is Dedupe) { // We're doing de-duping, therefore we have a primary key. // Cluster on the first 3 PK columns since BigQuery only allows up to 4 clustering // columns, // and we're always clustering on _airbyte_extracted_at - stream.primaryKey.stream().limit(3).forEach { columnId: ColumnId -> - clusterColumns.add(columnId.name) + (stream.importType as Dedupe).primaryKey.stream().limit(3).forEach { + pk: List -> + clusterColumns.add(columnNameMapping[pk.first()]!!) } } clusterColumns.add("_airbyte_extracted_at") diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt new file mode 100644 index 0000000000000..e528e1a3fbf45 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -0,0 +1,300 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.bigquery.typing_deduping + +import com.google.cloud.bigquery.BigQuery +import com.google.cloud.bigquery.Field +import com.google.cloud.bigquery.QueryJobConfiguration +import com.google.cloud.bigquery.StandardSQLTypeName +import com.google.cloud.bigquery.StandardTableDefinition +import com.google.cloud.bigquery.TableDefinition +import com.google.cloud.bigquery.TableId +import com.google.cloud.bigquery.TimePartitioning +import com.google.common.annotations.VisibleForTesting +import io.airbyte.cdk.integrations.base.JavaBaseConstants +import io.airbyte.cdk.load.command.Append +import io.airbyte.cdk.load.command.Dedupe +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.command.Overwrite +import io.airbyte.cdk.load.data.ObjectType +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.FinalTableInitialStatus +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.RawTableInitialStatus +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus +import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation +import io.airbyte.integrations.base.destination.typing_deduping.AlterTableReport +import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase +import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsIgnoreCase +import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.matchingKey +import io.github.oshai.kotlinlogging.KotlinLogging +import java.math.BigInteger +import java.util.stream.Collectors +import java.util.stream.Stream + +private val logger = KotlinLogging.logger {} + +class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : + DestinationInitialStatusGatherer { + private fun findExistingTable(finalTableName: TableName): TableDefinition? { + val table = bq.getTable(finalTableName.namespace, finalTableName.name) + return table?.getDefinition() + } + + private fun isFinalTableEmpty(finalTableName: TableName): Boolean { + return BigInteger.ZERO == + bq.getTable(TableId.of(finalTableName.namespace, finalTableName.name)).numRows + } + + private fun getInitialRawTableState( + rawTableName: TableName, + suffix: String + ): RawTableInitialStatus { + bq.getTable(TableId.of(rawTableName.namespace, rawTableName.name + suffix)) + ?: // Table doesn't exist. There are no unprocessed records, and no timestamp. + return RawTableInitialStatus( + hasUnprocessedRecords = false, + maxProcessedTimestamp = null, + ) + + val rawTableIdQuoted = """`${rawTableName.namespace}`.`${rawTableName.name}$suffix`""" + val unloadedRecordTimestamp = + bq.query( + QueryJobConfiguration.newBuilder( + """ + SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND) + FROM $rawTableIdQuoted + WHERE _airbyte_loaded_at IS NULL + """.trimIndent() + ) + .build() + ) + .iterateAll() + .iterator() + .next() + .first() + // If this value is null, then there are no records with null loaded_at. + // If it's not null, then we can return immediately - we've found some unprocessed records + // and their + // timestamp. + if (!unloadedRecordTimestamp.isNull) { + return RawTableInitialStatus( + hasUnprocessedRecords = true, + maxProcessedTimestamp = unloadedRecordTimestamp.timestampInstant, + ) + } + + val loadedRecordTimestamp = + bq.query( + QueryJobConfiguration.newBuilder( + """ + SELECT MAX(_airbyte_extracted_at) + FROM $rawTableIdQuoted + """.trimIndent() + ) + .build() + ) + .iterateAll() + .iterator() + .next() + .first() + // We know (from the previous query) that all records have been processed by T+D already. + // So we just need to get the timestamp of the most recent record. + return if (loadedRecordTimestamp.isNull) { + // Null timestamp because the table is empty. T+D can process the entire raw table + // during this sync. + RawTableInitialStatus(hasUnprocessedRecords = false, maxProcessedTimestamp = null) + } else { + // The raw table already has some records. T+D can skip all records with timestamp <= + // this value. + RawTableInitialStatus( + hasUnprocessedRecords = false, + maxProcessedTimestamp = loadedRecordTimestamp.timestampInstant + ) + } + } + + override suspend fun gatherInitialStatus( + streams: Map> + ): Map { + return streams.mapValues { (stream, names) -> + val (tableNames, columnNameMapping) = names + val finalTable = findExistingTable(tableNames.finalTableName!!) + val finalTableStatus = + finalTable?.let { + FinalTableInitialStatus( + isSchemaMismatch = + !existingSchemaMatchesStreamConfig( + stream, + columnNameMapping, + finalTable + ), + isEmpty = isFinalTableEmpty(tableNames.finalTableName!!), + // for now, just use 0. this means we will always use a temp final table. + // platform has a workaround for this, so it's OK. + // TODO only fetch this on truncate syncs + // TODO once we have destination state, use that instead of a query + finalTableGenerationId = 0, + ) + } + val rawTableState = getInitialRawTableState(tableNames.rawTableName!!, "") + val tempRawTableState = + getInitialRawTableState( + tableNames.rawTableName!!, + AbstractStreamOperation.TMP_TABLE_SUFFIX + ) + TypingDedupingDestinationInitialStatus( + finalTableStatus, + rawTableState, + tempRawTableState, + ) + } + } + + private fun existingSchemaMatchesStreamConfig( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition + ): Boolean { + val alterTableReport = buildAlterTableReport(stream, columnNameMapping, existingTable) + var tableClusteringMatches = false + var tablePartitioningMatches = false + if (existingTable is StandardTableDefinition) { + tableClusteringMatches = clusteringMatches(stream, columnNameMapping, existingTable) + tablePartitioningMatches = partitioningMatches(existingTable) + } + logger.info { + "Alter Table Report ${alterTableReport.columnsToAdd} ${alterTableReport.columnsToRemove} ${alterTableReport.columnsToChangeType}; Clustering $tableClusteringMatches; Partitioning $tablePartitioningMatches" + } + + return alterTableReport.isNoOp && tableClusteringMatches && tablePartitioningMatches + } + + private fun buildAlterTableReport( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition, + ): AlterTableReport { + val pks = getPks(stream, columnNameMapping) + + val streamSchema: Map = + (stream.schema as ObjectType).properties.entries.associate { + it.key to BigQuerySqlGenerator.toDialectType(it.value) + } + + val existingSchema = + existingTable.schema!!.fields.associate { it.name to it.type.standardType } + + // Columns in the StreamConfig that don't exist in the TableDefinition + val columnsToAdd = + streamSchema.keys + .stream() + .filter { name: String -> !containsIgnoreCase(existingSchema.keys, name) } + .collect(Collectors.toSet()) + + // Columns in the current schema that are no longer in the StreamConfig + val columnsToRemove = + existingSchema.keys + .stream() + .filter { name: String -> + !containsIgnoreCase(streamSchema.keys, name) && + !containsIgnoreCase(JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS, name) + } + .collect(Collectors.toSet()) + + // Columns that are typed differently than the StreamConfig + val columnsToChangeType = + Stream.concat( + streamSchema.keys + .stream() // If it's not in the existing schema, it should already be in the + // columnsToAdd Set + .filter { name: String -> + matchingKey( + existingSchema.keys, + name + ) // if it does exist, only include it in this set if the type (the + // value in each respective map) + // is different between the stream and existing schemas + .map { key: String -> + existingSchema[key] != streamSchema[name] + } // if there is no matching key, then don't include it because it + // is probably already in columnsToAdd + .orElse(false) + }, // OR columns that used to have a non-null constraint and shouldn't + // (https://github.com/airbytehq/airbyte/pull/31082) + + existingTable.schema!! + .fields + .stream() + .filter { field: Field -> pks.contains(field.name) } + .filter { field: Field -> field.mode == Field.Mode.REQUIRED } + .map { obj: Field -> obj.name } + ) + .collect(Collectors.toSet()) + + val isDestinationV2Format = schemaContainAllFinalTableV2AirbyteColumns(existingSchema.keys) + + return AlterTableReport( + columnsToAdd, + columnsToRemove, + columnsToChangeType, + isDestinationV2Format + ) + } + + @VisibleForTesting + fun clusteringMatches( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: StandardTableDefinition, + ): Boolean { + return (existingTable.clustering != null && + containsAllIgnoreCase( + HashSet(existingTable.clustering!!.fields), + BigQuerySqlGenerator.clusteringColumns(stream, columnNameMapping) + )) + } + + @VisibleForTesting + fun partitioningMatches(existingTable: StandardTableDefinition): Boolean { + return existingTable.timePartitioning != null && + existingTable.timePartitioning!! + .field + .equals("_airbyte_extracted_at", ignoreCase = true) && + TimePartitioning.Type.DAY == existingTable.timePartitioning!!.type + } + + /** + * Checks the schema to determine whether the table contains all expected final table airbyte + * columns + * + * @param columnNames the column names of the schema to check + * @return whether all the [JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS] are present + */ + @VisibleForTesting + fun schemaContainAllFinalTableV2AirbyteColumns(columnNames: Collection?): Boolean { + return JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.stream().allMatch { column: String? + -> + containsIgnoreCase(columnNames!!, column!!) + } + } + + private fun getPks( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping + ): Set { + return when (stream.importType) { + Append, + Overwrite -> emptySet() + is Dedupe -> + (stream.importType as Dedupe) + .primaryKey + .map { pk -> columnNameMapping[pk.first()]!! } + .toSet() + } + } +} From c3780682fc2e5e62e49275df0f3ba16814e2f32c Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 13:40:16 -0700 Subject: [PATCH 13/55] progress on sqlgenerator --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 16 +- .../load/orchestration/DestinationNames.kt | 11 +- .../TypingDedupingFinalTableOperations.kt | 12 +- .../TypingDedupingSqlGenerator.kt | 30 +- .../TypingDedupingStreamLoader.kt | 9 +- .../TypingDedupingWriter.kt | 13 +- .../typing_deduping/BigQuerySqlGenerator.kt | 264 ++++++++++-------- .../typing_deduping/BigqueryNameGenerators.kt | 49 ++++ 8 files changed, 264 insertions(+), 140 deletions(-) create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index eedd26a410ef0..96b36080e08a7 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -6,7 +6,15 @@ package io.airbyte.cdk.load.data import com.fasterxml.jackson.databind.JsonNode -sealed interface AirbyteType +sealed interface AirbyteType { + /** + * Utility method for database/warehouse destinations, which assume that the top-level schema is + * an object. + */ + fun getProperties(): LinkedHashMap { + return linkedMapOf() + } +} data object StringType : AirbyteType @@ -30,7 +38,11 @@ data class ArrayType(val items: FieldType) : AirbyteType data object ArrayTypeWithoutSchema : AirbyteType -data class ObjectType(val properties: LinkedHashMap) : AirbyteType +data class ObjectType(val properties: LinkedHashMap) : AirbyteType { + override fun getProperties(): LinkedHashMap { + return properties + } +} data object ObjectTypeWithEmptySchema : AirbyteType diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index 99c21d4f2730a..db3d635f2b690 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -36,7 +36,8 @@ data class TableNames( } data class TableName(val namespace: String, val name: String) { - fun prettyPrint() = "$namespace.$name" + fun prettyPrint(quote: String = "", suffix: String = "") = + "$quote$namespace$quote.$quote$name$suffix$quote" } fun TableName?.conflictsWith(other: TableName?): Boolean { @@ -63,15 +64,15 @@ fun interface ColumnNameGenerator { * In some database/warehouses, there's a difference between how a name is _displayed_, and how * the underlying engine actually treats it. For example, a column might be displayed as * `CamelCaseColumn`, but the engine actually treats it as lowercase `camelcasecolumn`, or - * truncate it to `CamelCas`. + * truncate it to `CamelCas`. Bigquery is an example of this: `create table foo (foo int, FOO + * int)` is invalid, because `foo` is duplicated. * * This is relevant for handling collisions between column names. We need to know what name will * be displayed to the user, since that's what we'll use in queries - but we also need to know * the "canonical" name to check whether two columns will collide. - * - * (edgao: I actually can't think of an example offhand. This logic predates me, and possibly - * doesn't need to exist.) */ data class ColumnName(val displayName: String, val canonicalName: String) fun getColumnName(column: String): ColumnName } + +const val CDC_DELETED_AT_COLUMN = "_ab_cdc_deleted_at" diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index ec8b523cebfb7..58c7dbcd495dc 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -5,6 +5,7 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationHandler import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.TableNames @@ -21,6 +22,7 @@ class TypingDedupingFinalTableOperations( fun createFinalTable( stream: DestinationStream, finalTableName: TableName, + columnNameMapping: ColumnNameMapping, finalTableSuffix: String, replace: Boolean ) { @@ -31,6 +33,7 @@ class TypingDedupingFinalTableOperations( sqlGenerator.createFinalTable( stream, finalTableName, + columnNameMapping, finalTableSuffix, replace = replace ) @@ -41,14 +44,18 @@ class TypingDedupingFinalTableOperations( fun softResetFinalTable( stream: DestinationStream, tableNames: TableNames, + columnNameMapping: ColumnNameMapping, ) { logger.info { "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" } - destinationHandler.execute(sqlGenerator.prepareTablesForSoftReset(stream, tableNames)) + destinationHandler.execute( + sqlGenerator.prepareTablesForSoftReset(stream, tableNames, columnNameMapping) + ) typeAndDedupe( stream, tableNames, + columnNameMapping, maxProcessedTimestamp = null, finalTableSuffix = SOFT_RESET_SUFFIX, ) @@ -85,6 +92,7 @@ class TypingDedupingFinalTableOperations( fun typeAndDedupe( stream: DestinationStream, tableNames: TableNames, + columnNameMapping: ColumnNameMapping, maxProcessedTimestamp: Instant?, finalTableSuffix: String ) { @@ -96,6 +104,7 @@ class TypingDedupingFinalTableOperations( sqlGenerator.updateFinalTable( stream, tableNames, + columnNameMapping, finalTableSuffix = finalTableSuffix, maxProcessedTimestamp = maxProcessedTimestamp, useExpensiveSaferCasting = false, @@ -110,6 +119,7 @@ class TypingDedupingFinalTableOperations( sqlGenerator.updateFinalTable( stream, tableNames, + columnNameMapping, finalTableSuffix = finalTableSuffix, maxProcessedTimestamp = maxProcessedTimestamp, useExpensiveSaferCasting = true, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index 5cb2f8f2feafe..eac44c872ec4e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -5,6 +5,7 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.Sql import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.TableNames @@ -27,6 +28,7 @@ interface TypingDedupingSqlGenerator { fun createFinalTable( stream: DestinationStream, tableName: TableName, + columnNameMapping: ColumnNameMapping, finalTableSuffix: String, replace: Boolean ): Sql @@ -69,6 +71,7 @@ interface TypingDedupingSqlGenerator { fun updateFinalTable( stream: DestinationStream, tableNames: TableNames, + columnNameMapping: ColumnNameMapping, finalTableSuffix: String, maxProcessedTimestamp: Instant?, useExpensiveSaferCasting: Boolean, @@ -87,15 +90,22 @@ interface TypingDedupingSqlGenerator { ): Sql fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql -} -/** Typically we need to create a soft reset temporary table and clear loaded at values */ -fun TypingDedupingSqlGenerator.prepareTablesForSoftReset( - stream: DestinationStream, - tableNames: TableNames -): Sql { - val createTempTable = - createFinalTable(stream, tableNames.finalTableName!!, SOFT_RESET_SUFFIX, replace = true) - val clearLoadedAt = clearLoadedAt(stream, tableNames.rawTableName!!) - return Sql.concat(createTempTable, clearLoadedAt) + /** Typically we need to create a soft reset temporary table and clear loaded at values */ + fun prepareTablesForSoftReset( + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + ): Sql { + val createTempTable = + createFinalTable( + stream, + tableNames.finalTableName!!, + columnNameMapping, + SOFT_RESET_SUFFIX, + replace = true + ) + val clearLoadedAt = clearLoadedAt(stream, tableNames.rawTableName!!) + return Sql.concat(createTempTable, clearLoadedAt) + } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 74f17f19c3546..02f004f3c4cdc 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -5,6 +5,7 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.TableNames.Companion.NO_SUFFIX import io.airbyte.cdk.load.orchestration.TableNames.Companion.TMP_TABLE_SUFFIX @@ -19,6 +20,7 @@ class TypingDedupingStreamLoader( override val stream: DestinationStream, private val initialStatus: TypingDedupingDestinationInitialStatus, private val tableNames: TableNames, + private val columnNameMapping: ColumnNameMapping, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, @@ -214,6 +216,7 @@ class TypingDedupingStreamLoader( finalTableOperations.createFinalTable( stream, tableNames.finalTableName!!, + columnNameMapping, NO_SUFFIX, replace = false ) @@ -252,7 +255,7 @@ class TypingDedupingStreamLoader( logger.info { "Truncate sync, and final table matches our generation, but has the wrong schema. Writing to it directly, but triggering a soft reset first." } - finalTableOperations.softResetFinalTable(stream, tableNames) + finalTableOperations.softResetFinalTable(stream, tableNames, columnNameMapping) return NO_SUFFIX } } else { @@ -266,7 +269,7 @@ class TypingDedupingStreamLoader( // Also, if a raw table migration wants us to do a soft reset, do that // here. logger.info { "Executing soft-reset on final table of stream ${stream.descriptor}" } - finalTableOperations.softResetFinalTable(stream, tableNames) + finalTableOperations.softResetFinalTable(stream, tableNames, columnNameMapping) } return NO_SUFFIX } @@ -281,6 +284,7 @@ class TypingDedupingStreamLoader( finalTableOperations.createFinalTable( stream, tableNames.finalTableName!!, + columnNameMapping, TMP_TABLE_SUFFIX, replace = true ) @@ -361,6 +365,7 @@ class TypingDedupingStreamLoader( finalTableOperations.typeAndDedupe( stream, tableNames, + columnNameMapping, maxProcessedTimestamp = maxProcessedTimestamp, finalTableSuffix = finalTmpTableSuffix ) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 142f8d38f80b3..5c7edd4a9317e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -6,7 +6,6 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.data.ObjectType import io.airbyte.cdk.load.orchestration.ColumnNameGenerator import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer @@ -46,7 +45,7 @@ class TypingDedupingWriter( ), ColumnNameMapping( // TODO handle collisions in column names - (stream.schema as ObjectType).properties.mapValues { (columnName, _) -> + stream.schema.getProperties().mapValues { (columnName, _) -> finalTableColumnNameGenerator.getColumnName(columnName).displayName } ) @@ -71,7 +70,12 @@ class TypingDedupingWriter( runBlocking(dispatcher) { streamsNeedingSoftReset.forEach { (stream, _) -> launch { - finalTableOperations.softResetFinalTable(stream, names[stream]!!.first) + val (tableNames, columnNameMapping) = names[stream]!! + finalTableOperations.softResetFinalTable( + stream, + tableNames, + columnNameMapping + ) } } } @@ -88,11 +92,12 @@ class TypingDedupingWriter( } override fun createStreamLoader(stream: DestinationStream): StreamLoader { - val tableNames = names[stream]!!.first + val (tableNames, columnNameMapping) = names[stream]!! return TypingDedupingStreamLoader( stream, initialStatuses[stream]!!, tableNames, + columnNameMapping, rawTableOperations, finalTableOperations, disableTypeDedupe = disableTypeDedupe, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 325e5c5bc792d..6849dd2629b7c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -7,57 +7,32 @@ import com.google.cloud.bigquery.StandardSQLTypeName import com.google.common.annotations.VisibleForTesting import io.airbyte.cdk.load.command.Dedupe import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.data.AirbyteType +import io.airbyte.cdk.load.orchestration.CDC_DELETED_AT_COLUMN import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.integrations.base.destination.typing_deduping.* +import io.airbyte.cdk.load.orchestration.Sql +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingSqlGenerator import io.airbyte.integrations.base.destination.typing_deduping.Array +import io.airbyte.integrations.base.destination.typing_deduping.ColumnId import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer import java.time.Instant import java.util.* import java.util.stream.Collectors import java.util.stream.Stream import org.apache.commons.lang3.StringUtils -import org.slf4j.Logger -import org.slf4j.LoggerFactory -class BigQuerySqlGenerator /** * @param projectId * @param datasetLocation This is technically redundant with [BigQueryDestinationHandler] setting * the query execution location, but let's be explicit since this is typically a compliance * requirement. */ -(private val projectId: String?, private val datasetLocation: String?) : SqlGenerator { - private val CDC_DELETED_AT_COLUMN = buildColumnId("_ab_cdc_deleted_at") - - private val LOGGER: Logger = LoggerFactory.getLogger(BigQuerySqlGenerator::class.java) - - override fun buildStreamId( - namespace: String, - name: String, - rawNamespaceOverride: String - ): StreamId { - return StreamId( - nameTransformer.getNamespace(namespace), - nameTransformer.convertStreamName(name), - nameTransformer.getNamespace(rawNamespaceOverride), - nameTransformer.convertStreamName(StreamId.concatenateRawTableName(namespace, name)), - namespace, - name - ) - } - - override fun buildColumnId(name: String, suffix: String?): ColumnId { - val nameWithSuffix = name + suffix - return ColumnId( - nameTransformer.getIdentifier(nameWithSuffix), - name, // Bigquery columns are case-insensitive, so do all our validation on the - // lowercased name - nameTransformer.getIdentifier(nameWithSuffix.lowercase(Locale.getDefault())) - ) - } - +class BigQuerySqlGenerator(private val projectId: String?, private val datasetLocation: String?) : + TypingDedupingSqlGenerator { private fun extractAndCast( - column: ColumnId, + columnName: String, airbyteType: AirbyteType, forceSafeCast: Boolean ): String { @@ -134,15 +109,21 @@ class BigQuerySqlGenerator } } - override fun createTable(stream: StreamConfig, suffix: String, force: Boolean): Sql { + override fun createFinalTable( + stream: DestinationStream, + tableName: TableName, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + replace: Boolean + ): Sql { val columnDeclarations = columnsAndTypes(stream) val clusterConfig = - clusteringColumns(stream) + clusteringColumns(stream, columnNameMapping) .stream() .map { c: String? -> StringUtils.wrap(c, QUOTE) } .collect(Collectors.joining(", ")) - val forceCreateTable = if (force) "OR REPLACE" else "" - val finalTableId = stream.id.finalTableId(QUOTE, suffix) + val forceCreateTable = if (replace) "OR REPLACE" else "" + val finalTableId = tableName.prettyPrint(QUOTE, finalTableSuffix) return Sql.of( """ CREATE $forceCreateTable TABLE `$projectId`.$finalTableId ( @@ -158,75 +139,117 @@ class BigQuerySqlGenerator ) } - private fun columnsAndTypes(stream: StreamConfig): String { - return stream.columns.entries - .stream() - .map { column: Map.Entry -> - java.lang.String.join(" ", column.key.name(QUOTE), toDialectType(column.value).name) + private fun columnsAndTypes( + stream: DestinationStream, + columnNameMapping: ColumnNameMapping + ): String { + return stream.schema + .getProperties() + .map { (fieldName, type) -> + val columnName = columnNameMapping[fieldName]!! + val typeName = toDialectType(type).name + "`$columnName` $typeName" } - .collect(Collectors.joining(",\n")) + .joinToString(",\n") } - override fun prepareTablesForSoftReset(stream: StreamConfig): Sql { + override fun prepareTablesForSoftReset( + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + ): Sql { // Bigquery can't run DDL in a transaction, so these are separate transactions. - return Sql - .concat( // If a previous sync failed to delete the soft reset temp table (unclear why - // this happens), - // AND this sync is trying to change the clustering config, then we need to manually - // drop the soft - // reset temp table. - // Even though we're using CREATE OR REPLACE TABLE, bigquery will still complain - // about the - // clustering config being changed. - // So we explicitly drop the soft reset temp table first. - dropTableIfExists(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX), - createTable(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX, true), - clearLoadedAt(stream.id) - ) + return Sql.concat( + // If a previous sync failed to delete the soft reset temp table (unclear why + // this happens), + // AND this sync is trying to change the clustering config, then we need to manually + // drop the soft + // reset temp table. + // Even though we're using CREATE OR REPLACE TABLE, bigquery will still complain + // about the + // clustering config being changed. + // So we explicitly drop the soft reset temp table first. + dropTableIfExists(tableNames.finalTableName!!, TableNames.SOFT_RESET_SUFFIX), + createFinalTable( + stream, + tableNames.finalTableName!!, + columnNameMapping, + TableNames.SOFT_RESET_SUFFIX, + true + ), + clearLoadedAt(stream, tableNames.rawTableName!!) + ) } - private fun dropTableIfExists(stream: StreamConfig, suffix: String): Sql { - val tableId = stream.id.finalTableId(QUOTE, suffix) + private fun dropTableIfExists( + finalTableName: TableName, + suffix: String, + ): Sql { + val tableId = finalTableName.prettyPrint(QUOTE, suffix) return Sql.of("""DROP TABLE IF EXISTS `$projectId`.$tableId;""") } - override fun clearLoadedAt(streamId: StreamId): Sql { - val rawTableId = streamId.rawTableId(QUOTE) + override fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql { + val rawTableId = rawTableName.prettyPrint(QUOTE) return Sql.of( """UPDATE `$projectId`.$rawTableId SET _airbyte_loaded_at = NULL WHERE 1=1;""" ) } - override fun updateTable( - stream: StreamConfig, - finalSuffix: String, - minRawTimestamp: Optional, - useExpensiveSaferCasting: Boolean + override fun updateFinalTable( + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + maxProcessedTimestamp: Instant?, + useExpensiveSaferCasting: Boolean, ): Sql { val handleNewRecords = - if (stream.postImportAction == ImportType.DEDUPE) { - upsertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp) + if (stream.importType is Dedupe) { + upsertNewRecords( + stream, + tableNames, + columnNameMapping, + finalTableSuffix, + useExpensiveSaferCasting, + maxProcessedTimestamp + ) } else { - insertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp) + insertNewRecords( + stream, + tableNames, + columnNameMapping, + finalTableSuffix, + useExpensiveSaferCasting, + maxProcessedTimestamp + ) } - val commitRawTable = commitRawTable(stream.id, minRawTimestamp) + val commitRawTable = commitRawTable(tableNames.rawTableName!!, maxProcessedTimestamp) return Sql.transactionally(handleNewRecords, commitRawTable) } private fun insertNewRecords( - stream: StreamConfig, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, finalSuffix: String, forceSafeCasting: Boolean, - minRawTimestamp: Optional + minRawTimestamp: Instant?, ): String { val columnList: String = - stream.columns.keys + stream.schema + .getProperties() + .keys .stream() - .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } + .map { fieldName -> + val columnName = columnNameMapping[fieldName]!! + "`$columnName`," + } .collect(Collectors.joining("\n")) - val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp) - val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix) + val extractNewRawRecords = + extractNewRawRecords(stream, tableNames, forceSafeCasting, minRawTimestamp) + val finalTableId = tableNames.finalTableName!!.prettyPrint(QUOTE, finalSuffix) return """ INSERT INTO `$projectId`.$finalTableId @@ -242,10 +265,12 @@ class BigQuerySqlGenerator } private fun upsertNewRecords( - stream: StreamConfig, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, finalSuffix: String, forceSafeCasting: Boolean, - minRawTimestamp: Optional + minRawTimestamp: Instant?, ): String { val pkEquivalent = stream.primaryKey @@ -351,27 +376,30 @@ class BigQuerySqlGenerator * dedupes the records (since we only need the most-recent record to upsert). */ private fun extractNewRawRecords( - stream: StreamConfig, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, forceSafeCasting: Boolean, - minRawTimestamp: Optional + minRawTimestamp: Instant?, ): String { val columnCasts: String = - stream.columns.entries - .stream() - .map { col: Map.Entry -> - val extractAndCast = extractAndCast(col.key, col.value, forceSafeCasting) - val columnName = col.key.name(QUOTE) - """$extractAndCast as $columnName,""" + stream.schema + .getProperties() + .map { (fieldName, type) -> + val columnName = columnNameMapping[fieldName]!! + val extractAndCast = extractAndCast(columnName, type.type, forceSafeCasting) + "$extractAndCast as `$columnName`," } - .collect(Collectors.joining("\n")) + .joinToString("\n") val columnErrors = if (forceSafeCasting) { "[" + - stream.columns.entries - .stream() - .map { col: Map.Entry -> - val rawColName = escapeColumnNameForJsonPath(col.key.originalName) - val jsonExtract = extractAndCast(col.key, col.value, true) + stream.schema + .getProperties() + .map { (fieldName, type) -> + val columnName = columnNameMapping[fieldName]!! + val rawColName = escapeColumnNameForJsonPath(fieldName) + val jsonExtract = extractAndCast(columnName, type.type, true) // Explicitly parse json here. This is safe because // we're not using the actual value anywhere, // and necessary because json_query @@ -385,7 +413,7 @@ class BigQuerySqlGenerator END """.trimIndent() } - .collect(Collectors.joining(",\n")) + + .joinToString(",\n") + "]" } else { // We're not safe casting, so any error should throw an exception and trigger the @@ -394,14 +422,15 @@ class BigQuerySqlGenerator } val columnList: String = - stream.columns.keys - .stream() - .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } - .collect(Collectors.joining("\n")) + stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + val columnName = columnNameMapping[fieldName]!! + "`$columnName`," + } val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) - val rawTableId = stream.id.rawTableId(QUOTE) - if (stream.postImportAction == ImportType.DEDUPE) { + val rawTableId = tableNames.rawTableName!!.prettyPrint(QUOTE) + if (stream.importType is Dedupe) { + val importType = stream.importType as Dedupe // When deduping, we need to dedup the raw records. Note the row_number() invocation in // the SQL // statement. Do the same extract+cast CTE + airbyte_meta construction as in non-dedup @@ -414,7 +443,7 @@ class BigQuerySqlGenerator // out-of-order records. var cdcConditionalOrIncludeStatement = "" - if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) { + if (stream.schema.getProperties().containsKey(CDC_DELETED_AT_COLUMN)) { cdcConditionalOrIncludeStatement = """ OR ( @@ -425,13 +454,17 @@ class BigQuerySqlGenerator } val pkList = - stream.primaryKey - .stream() - .map { columnId: ColumnId -> columnId.name(QUOTE) } - .collect(Collectors.joining(",")) + importType.primaryKey.joinToString(",") { fieldName -> + val columnName = columnNameMapping[fieldName.first()]!! + "`$columnName`" + } val cursorOrderClause = - stream.cursor - .map { cursorId: ColumnId -> cursorId.name(QUOTE) + " DESC NULLS LAST," } + importType.cursor + .first() + .map { fieldName -> + val columnName = columnNameMapping[fieldName]!! + "`$columnName` DESC NULLS LAST" + } .orElse("") return """ @@ -513,8 +546,8 @@ class BigQuerySqlGenerator } @VisibleForTesting - fun commitRawTable(id: StreamId, minRawTimestamp: Optional): String { - val rawTableId = id.rawTableId(QUOTE) + fun commitRawTable(rawTableName: TableName, minRawTimestamp: Instant?): String { + val rawTableId = rawTableName.prettyPrint(QUOTE) val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) return """ UPDATE `$projectId`.$rawTableId @@ -609,7 +642,7 @@ class BigQuerySqlGenerator companion object { const val QUOTE: String = "`" - private val nameTransformer = BigQuerySQLNameTransformer() + val nameTransformer = BigQuerySQLNameTransformer() @JvmStatic fun toDialectType(type: AirbyteType): StandardSQLTypeName { @@ -673,10 +706,9 @@ class BigQuerySqlGenerator return clusterColumns } - private fun buildExtractedAtCondition(minRawTimestamp: Optional): String { - return minRawTimestamp - .map { ts: Instant -> " AND _airbyte_extracted_at > '$ts'" } - .orElse("") + private fun buildExtractedAtCondition(minRawTimestamp: Instant?): String { + return minRawTimestamp?.let { ts: Instant -> " AND _airbyte_extracted_at > '$ts'" } + ?: "" } private fun cast(content: String, asType: String, useSafeCast: Boolean): String { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt new file mode 100644 index 0000000000000..3a6e9d9153833 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt @@ -0,0 +1,49 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.integrations.destination.bigquery.typing_deduping + +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameGenerator +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNameGenerator +import io.airbyte.integrations.base.destination.typing_deduping.StreamId +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer +import java.util.Locale + +class BigqueryRawTableNameGenerator( + private val defaultNamespace: String, + private val rawNamespace: String, +) : TableNameGenerator { + override fun getTableName(streamDescriptor: DestinationStream.Descriptor): TableName { + return TableName( + nameTransformer.getNamespace(rawNamespace), + nameTransformer.convertStreamName( + StreamId.concatenateRawTableName( + streamDescriptor.namespace ?: defaultNamespace, + streamDescriptor.name + ) + ), + ) + } +} + +class BigqueryFinalTableNameGenerator(private val defaultNamespace: String) : TableNameGenerator { + override fun getTableName(streamDescriptor: DestinationStream.Descriptor) = + TableName( + nameTransformer.getNamespace(streamDescriptor.namespace ?: defaultNamespace), + nameTransformer.convertStreamName(streamDescriptor.name), + ) +} + +class BigqueryColumnNameGenerator : ColumnNameGenerator { + override fun getColumnName(column: String): ColumnNameGenerator.ColumnName { + return ColumnNameGenerator.ColumnName( + nameTransformer.getIdentifier(column), + // Bigquery columns are case-insensitive, so do all our validation on the + // lowercased name + nameTransformer.getIdentifier(column.lowercase(Locale.getDefault())), + ) + } +} From 7ffb0b823c06ec33d7d40500ef2e489ebf68c38e Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 14:47:12 -0700 Subject: [PATCH 14/55] progress --- .../cdk/load/command/DestinationStream.kt | 4 + .../TypingDedupingSqlGenerator.kt | 2 +- .../typing_deduping/BigQuerySqlGenerator.kt | 213 ++++++++---------- 3 files changed, 98 insertions(+), 121 deletions(-) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt index f598d59827ca8..b634a636cbab7 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/command/DestinationStream.kt @@ -7,6 +7,7 @@ package io.airbyte.cdk.load.command import io.airbyte.cdk.load.data.AirbyteType import io.airbyte.cdk.load.data.json.AirbyteTypeToJsonSchema import io.airbyte.cdk.load.data.json.JsonSchemaToAirbyteType +import io.airbyte.cdk.load.message.DestinationRecord import io.airbyte.protocol.models.v0.AirbyteStream import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream import io.airbyte.protocol.models.v0.DestinationSyncMode @@ -128,6 +129,9 @@ data class Dedupe( /** * theoretically, the path to the cursor. In practice, most destinations only support cursors at * the root level, i.e. `listOf(cursorField)`. + * + * If this is set to an empty list, then the destination should use + * [DestinationRecord.message.record.emittedAt] as the cursor. */ val cursor: List, ) : ImportType diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index eac44c872ec4e..288b5f105d1b9 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -86,7 +86,7 @@ interface TypingDedupingSqlGenerator { fun overwriteFinalTable( stream: DestinationStream, finalTableName: TableName, - finalTableSuffix: String + finalTableSuffix: String, ): Sql fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 6849dd2629b7c..2b223cc685320 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -8,6 +8,22 @@ import com.google.common.annotations.VisibleForTesting import io.airbyte.cdk.load.command.Dedupe import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.data.AirbyteType +import io.airbyte.cdk.load.data.ArrayType +import io.airbyte.cdk.load.data.ArrayTypeWithoutSchema +import io.airbyte.cdk.load.data.BooleanType +import io.airbyte.cdk.load.data.DateType +import io.airbyte.cdk.load.data.IntegerType +import io.airbyte.cdk.load.data.NumberType +import io.airbyte.cdk.load.data.ObjectType +import io.airbyte.cdk.load.data.ObjectTypeWithEmptySchema +import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema +import io.airbyte.cdk.load.data.StringType +import io.airbyte.cdk.load.data.TimeTypeWithTimezone +import io.airbyte.cdk.load.data.TimeTypeWithoutTimezone +import io.airbyte.cdk.load.data.TimestampTypeWithTimezone +import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone +import io.airbyte.cdk.load.data.UnionType +import io.airbyte.cdk.load.data.UnknownType import io.airbyte.cdk.load.orchestration.CDC_DELETED_AT_COLUMN import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.Sql @@ -15,7 +31,6 @@ import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingSqlGenerator import io.airbyte.integrations.base.destination.typing_deduping.Array -import io.airbyte.integrations.base.destination.typing_deduping.ColumnId import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer import java.time.Instant import java.util.* @@ -116,7 +131,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo finalTableSuffix: String, replace: Boolean ): Sql { - val columnDeclarations = columnsAndTypes(stream) + val columnDeclarations = columnsAndTypes(stream, columnNameMapping) val clusterConfig = clusteringColumns(stream, columnNameMapping) .stream() @@ -147,7 +162,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo .getProperties() .map { (fieldName, type) -> val columnName = columnNameMapping[fieldName]!! - val typeName = toDialectType(type).name + val typeName = toDialectType(type.type).name "`$columnName` $typeName" } .joinToString(",\n") @@ -248,7 +263,13 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } .collect(Collectors.joining("\n")) val extractNewRawRecords = - extractNewRawRecords(stream, tableNames, forceSafeCasting, minRawTimestamp) + extractNewRawRecords( + stream, + tableNames, + columnNameMapping, + forceSafeCasting, + minRawTimestamp + ) val finalTableId = tableNames.finalTableName!!.prettyPrint(QUOTE, finalSuffix) return """ @@ -272,32 +293,38 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo forceSafeCasting: Boolean, minRawTimestamp: Instant?, ): String { + val importType = stream.importType as Dedupe val pkEquivalent = - stream.primaryKey - .stream() - .map { pk: ColumnId -> - val quotedPk = pk.name(QUOTE) - ("""(target_table.$quotedPk = new_record.$quotedPk OR (target_table.$quotedPk IS NULL AND new_record.$quotedPk IS NULL))""") - } - .collect(Collectors.joining(" AND ")) + importType.primaryKey.joinToString(" AND ") { fieldPath -> + val fieldName = fieldPath.first() + val columnName = columnNameMapping[fieldName]!! + """(target_table.`$columnName` = new_record.`$columnName` OR (target_table.`$columnName` IS NULL AND new_record.`$columnName` IS NULL))""" + } val columnList: String = - stream.columns.keys - .stream() - .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } - .collect(Collectors.joining("\n")) + stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + val columnName = columnNameMapping[fieldName]!! + "`$columnName`," + } val newRecordColumnList: String = - stream.columns.keys - .stream() - .map { quotedColumnId: ColumnId -> - "new_record." + quotedColumnId.name(QUOTE) + "," - } - .collect(Collectors.joining("\n")) - val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp) + stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + val columnName = columnNameMapping[fieldName]!! + "new_record.`$columnName`," + } + val extractNewRawRecords = + extractNewRawRecords( + stream, + tableNames, + columnNameMapping, + forceSafeCasting, + minRawTimestamp + ) val cursorComparison: String - if (stream.cursor.isPresent) { - val cursor = stream.cursor.get().name(QUOTE) + if (importType.cursor.isNotEmpty()) { + val cursorFieldName = importType.cursor.first() + val cursorColumnName = columnNameMapping[cursorFieldName]!! + val cursor = "`$cursorColumnName`" // Build a condition for "new_record is more recent than target_table": cursorComparison = // First, compare the cursors. (""" @@ -316,7 +343,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo val cdcDeleteClause: String val cdcSkipInsertClause: String - if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) { + if (stream.schema.getProperties().containsKey(CDC_DELETED_AT_COLUMN)) { // Execute CDC deletions if there's already a record cdcDeleteClause = "WHEN MATCHED AND new_record._ab_cdc_deleted_at IS NOT NULL AND $cursorComparison THEN DELETE" @@ -330,14 +357,11 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } val columnAssignments: String = - stream.columns.keys - .stream() - .map { airbyteType: ColumnId -> - val column = airbyteType.name(QUOTE) - "$column = new_record.$column," - } - .collect(Collectors.joining("\n")) - val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix) + stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + val column = columnNameMapping[fieldName]!! + "`$column` = new_record.`$column`," + } + val finalTableId = tableNames.finalTableName!!.prettyPrint(QUOTE, finalSuffix) return """ MERGE `$projectId`.$finalTableId target_table @@ -459,13 +483,16 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo "`$columnName`" } val cursorOrderClause = - importType.cursor - .first() - .map { fieldName -> - val columnName = columnNameMapping[fieldName]!! - "`$columnName` DESC NULLS LAST" - } - .orElse("") + if (importType.cursor.isEmpty()) { + "" + } else if (importType.cursor.size == 1) { + val columnName = importType.cursor.first() + "`$columnName` DESC NULLS LAST" + } else { + throw UnsupportedOperationException( + "Only top-level cursors are supported, got ${importType.cursor}" + ) + } return """ WITH intermediate_data AS ( @@ -558,13 +585,16 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo """.trimIndent() } - override fun overwriteFinalTable(stream: StreamId, finalSuffix: String): Sql { - val finalTableId = stream.finalTableId(QUOTE) - val tempFinalTableId = stream.finalTableId(QUOTE, finalSuffix) - val realFinalTableName = stream.finalName(QUOTE) + override fun overwriteFinalTable( + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String, + ): Sql { + val finalTableId = finalTableName.prettyPrint(QUOTE) + val tempFinalTableId = finalTableName.prettyPrint(QUOTE, finalTableSuffix) return Sql.separately( "DROP TABLE IF EXISTS `$projectId`.$finalTableId;", - "ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO $realFinalTableName;" + "ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO `${finalTableName.name}`;" ) } @@ -574,43 +604,6 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo .collect(Collectors.joining(".")) } - override fun createSchema(schema: String): Sql { - val projectId = StringUtils.wrap(projectId, QUOTE) - val quotedSchema = StringUtils.wrap(schema, QUOTE) - return Sql.of( - """CREATE SCHEMA IF NOT EXISTS $projectId.$quotedSchema OPTIONS(location="$datasetLocation");""" - ) - } - - override fun migrateFromV1toV2(streamId: StreamId, namespace: String, tableName: String): Sql { - val v2RawTable = streamId.rawTableId(QUOTE) - val v1RawTable = wrapAndQuote(namespace, tableName) - return Sql.of( - """ - CREATE OR REPLACE TABLE `$projectId`.$v2RawTable ( - _airbyte_raw_id STRING, - _airbyte_data STRING, - _airbyte_extracted_at TIMESTAMP, - _airbyte_loaded_at TIMESTAMP, - _airbyte_meta STRING, - _airbyte_generation_id INTEGER - ) - PARTITION BY DATE(_airbyte_extracted_at) - CLUSTER BY _airbyte_extracted_at - AS ( - SELECT - _airbyte_ab_id AS _airbyte_raw_id, - _airbyte_data AS _airbyte_data, - _airbyte_emitted_at AS _airbyte_extracted_at, - CAST(NULL AS TIMESTAMP) AS _airbyte_loaded_at, - '{"sync_id": 0, "changes": []}' AS _airbyte_meta, - 0 as _airbyte_generation_id - FROM `$projectId`.$v1RawTable - ); - """.trimIndent() - ) - } - /** * Does two things: escape single quotes (for use inside sql string literals),and escape double * quotes (for use inside JSON paths). For example, if a column name is foo'bar"baz, then we @@ -646,44 +639,24 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo @JvmStatic fun toDialectType(type: AirbyteType): StandardSQLTypeName { - // switch pattern-matching is still in preview at language level 17 :( - if (type is AirbyteProtocolType) { - return toDialectType(type) - } else if (type is Struct) { - return StandardSQLTypeName.JSON - } else if (type is Array) { - return StandardSQLTypeName.JSON - } else if (type is UnsupportedOneOf) { - return StandardSQLTypeName.JSON - } else if (type is Union) { - val typeWithPrecedence: AirbyteType = type.chooseType() - val dialectType: StandardSQLTypeName - if ((typeWithPrecedence is Struct) || (typeWithPrecedence is Array)) { - dialectType = StandardSQLTypeName.JSON - } else { - dialectType = toDialectType(typeWithPrecedence as AirbyteProtocolType) - } - return dialectType - } - - // Literally impossible; AirbyteType is a sealed interface. - throw IllegalArgumentException("Unsupported AirbyteType: $type") - } - - // TODO maybe make this a BiMap and elevate this method and its inverse - // (toDestinationSQLType?) to the SQLGenerator? - fun toDialectType(airbyteProtocolType: AirbyteProtocolType): StandardSQLTypeName { - return when (airbyteProtocolType) { - AirbyteProtocolType.STRING, - AirbyteProtocolType.TIME_WITH_TIMEZONE -> StandardSQLTypeName.STRING - AirbyteProtocolType.NUMBER -> StandardSQLTypeName.NUMERIC - AirbyteProtocolType.INTEGER -> StandardSQLTypeName.INT64 - AirbyteProtocolType.BOOLEAN -> StandardSQLTypeName.BOOL - AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE -> StandardSQLTypeName.TIMESTAMP - AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE -> StandardSQLTypeName.DATETIME - AirbyteProtocolType.TIME_WITHOUT_TIMEZONE -> StandardSQLTypeName.TIME - AirbyteProtocolType.DATE -> StandardSQLTypeName.DATE - AirbyteProtocolType.UNKNOWN -> StandardSQLTypeName.JSON + return when (type) { + BooleanType -> StandardSQLTypeName.BOOL + DateType -> StandardSQLTypeName.DATE + IntegerType -> StandardSQLTypeName.INT64 + NumberType -> StandardSQLTypeName.NUMERIC + StringType -> StandardSQLTypeName.STRING + TimeTypeWithTimezone -> StandardSQLTypeName.STRING + TimeTypeWithoutTimezone -> StandardSQLTypeName.TIME + TimestampTypeWithTimezone -> StandardSQLTypeName.TIMESTAMP + TimestampTypeWithoutTimezone -> StandardSQLTypeName.DATETIME + is ArrayType, + ArrayTypeWithoutSchema, + is ObjectType, + ObjectTypeWithEmptySchema, + ObjectTypeWithoutSchema, + // TODO handle LegacyUnion type (do the Union.chooseType thing) + is UnionType, + is UnknownType -> StandardSQLTypeName.JSON } } From 4694dea03ba7aedcd18208e8473ba2a252d48f6e Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Tue, 15 Apr 2025 14:53:41 -0700 Subject: [PATCH 15/55] mostly done? --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 23 +++++++-- .../typing_deduping/BigQuerySqlGenerator.kt | 50 ++++++++----------- 2 files changed, 40 insertions(+), 33 deletions(-) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index 96b36080e08a7..3bcf32ebd1e9e 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -14,6 +14,11 @@ sealed interface AirbyteType { fun getProperties(): LinkedHashMap { return linkedMapOf() } + + val isObject: Boolean + get() = false + val isArray: Boolean + get() = false } data object StringType : AirbyteType @@ -34,19 +39,29 @@ data object TimeTypeWithTimezone : AirbyteType data object TimeTypeWithoutTimezone : AirbyteType -data class ArrayType(val items: FieldType) : AirbyteType +data class ArrayType(val items: FieldType) : AirbyteType { + override val isArray = true +} -data object ArrayTypeWithoutSchema : AirbyteType +data object ArrayTypeWithoutSchema : AirbyteType { + override val isArray = true +} data class ObjectType(val properties: LinkedHashMap) : AirbyteType { override fun getProperties(): LinkedHashMap { return properties } + + override val isObject = true } -data object ObjectTypeWithEmptySchema : AirbyteType +data object ObjectTypeWithEmptySchema : AirbyteType { + override val isObject = true +} -data object ObjectTypeWithoutSchema : AirbyteType +data object ObjectTypeWithoutSchema : AirbyteType { + override val isObject = true +} data class UnionType( val options: Set, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 2b223cc685320..6931968b968ff 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -30,12 +30,10 @@ import io.airbyte.cdk.load.orchestration.Sql import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingSqlGenerator -import io.airbyte.integrations.base.destination.typing_deduping.Array import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer import java.time.Instant import java.util.* import java.util.stream.Collectors -import java.util.stream.Stream import org.apache.commons.lang3.StringUtils /** @@ -51,14 +49,15 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo airbyteType: AirbyteType, forceSafeCast: Boolean ): String { - if (airbyteType is Union) { + // TODO this should actually be LegacyUnionType + if (airbyteType is UnionType) { // This is guaranteed to not be a Union, so we won't recurse infinitely val chosenType: AirbyteType = airbyteType.chooseType() - return extractAndCast(column, chosenType, forceSafeCast) + return extractAndCast(columnName, chosenType, forceSafeCast) } - val columnName = escapeColumnNameForJsonPath(column.originalName) + val jsonPathEscapedColumnName = escapeColumnNameForJsonPath(columnName) - if (airbyteType is Struct) { + if (airbyteType.isObject) { // We need to validate that the struct is actually a struct. // Note that struct columns are actually nullable in two ways. For a column `foo`: // {foo: null} and {} are both valid, and are both written to the final table as a SQL @@ -68,53 +67,52 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo // JSON_QUERY(JSON'{"foo": null}', '$."foo"') returns a JSON null. return """ PARSE_JSON(CASE - WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL - OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'object' + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"'), wide_number_mode=>'round')) != 'object' THEN NULL - ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') END, wide_number_mode=>'round') """.trimIndent() } - if (airbyteType is Array) { + if (airbyteType.isArray) { // Much like the Struct case above, arrays need special handling. return """ PARSE_JSON(CASE - WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL - OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'array' + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"'), wide_number_mode=>'round')) != 'array' THEN NULL - ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') END, wide_number_mode=>'round') """.trimIndent() } - if (airbyteType is UnsupportedOneOf || airbyteType === AirbyteProtocolType.UNKNOWN) { + if (airbyteType is UnionType || airbyteType is UnknownType) { // JSON_QUERY returns a SQL null if the field contains a JSON null, so we actually parse // the // airbyte_data to json // and json_query it directly (which preserves nulls correctly). - return """JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$columnName"')""" + return """JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$jsonPathEscapedColumnName"')""" } - if (airbyteType === AirbyteProtocolType.STRING) { + if (airbyteType is StringType) { // Special case String to only use json value for type string and parse the json for // others // Naive json_value returns NULL for object/array values and json_query adds escaped - // quotes to the - // string. + // quotes to the string. return """ (CASE - WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL - OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'string' - THEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"'), wide_number_mode=>'round')) != 'string' + THEN JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') ELSE - JSON_VALUE(`_airbyte_data`, '${'$'}."$columnName"') + JSON_VALUE(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') END) """.trimIndent() } val dialectType = toDialectType(airbyteType) - val baseTyping = """JSON_VALUE(`_airbyte_data`, '$."$columnName"')""" + val baseTyping = """JSON_VALUE(`_airbyte_data`, '$."$jsonPathEscapedColumnName"')""" return if (dialectType == StandardSQLTypeName.STRING) { // json_value implicitly returns a string, so we don't need to cast it. baseTyping @@ -598,12 +596,6 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo ) } - private fun wrapAndQuote(namespace: String, tableName: String): String { - return Stream.of(namespace, tableName) - .map { part: String? -> StringUtils.wrap(part, QUOTE) } - .collect(Collectors.joining(".")) - } - /** * Does two things: escape single quotes (for use inside sql string literals),and escape double * quotes (for use inside JSON paths). For example, if a column name is foo'bar"baz, then we From 31da2f9fe8bcce16ff13e41c2a4cefc2d28f7761 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 16 Apr 2025 16:11:03 -0700 Subject: [PATCH 16/55] ffs --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 4 ++-- .../TypingDedupingWriter.kt | 2 +- .../typing_deduping/BigQuerySqlGenerator.kt | 20 +++++++++---------- 3 files changed, 13 insertions(+), 13 deletions(-) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index 3bcf32ebd1e9e..f1a8883d7b111 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -11,7 +11,7 @@ sealed interface AirbyteType { * Utility method for database/warehouse destinations, which assume that the top-level schema is * an object. */ - fun getProperties(): LinkedHashMap { + fun asColumns(): LinkedHashMap { return linkedMapOf() } @@ -48,7 +48,7 @@ data object ArrayTypeWithoutSchema : AirbyteType { } data class ObjectType(val properties: LinkedHashMap) : AirbyteType { - override fun getProperties(): LinkedHashMap { + override fun asColumns(): LinkedHashMap { return properties } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 5c7edd4a9317e..a4db18f1dc50a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -45,7 +45,7 @@ class TypingDedupingWriter( ), ColumnNameMapping( // TODO handle collisions in column names - stream.schema.getProperties().mapValues { (columnName, _) -> + stream.schema.asColumns().mapValues { (columnName, _) -> finalTableColumnNameGenerator.getColumnName(columnName).displayName } ) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 6931968b968ff..3d12658f165f7 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -157,7 +157,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo columnNameMapping: ColumnNameMapping ): String { return stream.schema - .getProperties() + .asColumns() .map { (fieldName, type) -> val columnName = columnNameMapping[fieldName]!! val typeName = toDialectType(type.type).name @@ -252,7 +252,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo ): String { val columnList: String = stream.schema - .getProperties() + .asColumns() .keys .stream() .map { fieldName -> @@ -300,12 +300,12 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } val columnList: String = - stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + stream.schema.asColumns().keys.joinToString("\n") { fieldName -> val columnName = columnNameMapping[fieldName]!! "`$columnName`," } val newRecordColumnList: String = - stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + stream.schema.asColumns().keys.joinToString("\n") { fieldName -> val columnName = columnNameMapping[fieldName]!! "new_record.`$columnName`," } @@ -341,7 +341,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo val cdcDeleteClause: String val cdcSkipInsertClause: String - if (stream.schema.getProperties().containsKey(CDC_DELETED_AT_COLUMN)) { + if (stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN)) { // Execute CDC deletions if there's already a record cdcDeleteClause = "WHEN MATCHED AND new_record._ab_cdc_deleted_at IS NOT NULL AND $cursorComparison THEN DELETE" @@ -355,7 +355,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } val columnAssignments: String = - stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + stream.schema.asColumns().keys.joinToString("\n") { fieldName -> val column = columnNameMapping[fieldName]!! "`$column` = new_record.`$column`," } @@ -406,7 +406,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo ): String { val columnCasts: String = stream.schema - .getProperties() + .asColumns() .map { (fieldName, type) -> val columnName = columnNameMapping[fieldName]!! val extractAndCast = extractAndCast(columnName, type.type, forceSafeCasting) @@ -417,7 +417,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo if (forceSafeCasting) { "[" + stream.schema - .getProperties() + .asColumns() .map { (fieldName, type) -> val columnName = columnNameMapping[fieldName]!! val rawColName = escapeColumnNameForJsonPath(fieldName) @@ -444,7 +444,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } val columnList: String = - stream.schema.getProperties().keys.joinToString("\n") { fieldName -> + stream.schema.asColumns().keys.joinToString("\n") { fieldName -> val columnName = columnNameMapping[fieldName]!! "`$columnName`," } @@ -465,7 +465,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo // out-of-order records. var cdcConditionalOrIncludeStatement = "" - if (stream.schema.getProperties().containsKey(CDC_DELETED_AT_COLUMN)) { + if (stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN)) { cdcConditionalOrIncludeStatement = """ OR ( From 1f6f2c8624564b509381030cbb6e4ad028133d41 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 16 Apr 2025 16:39:03 -0700 Subject: [PATCH 17/55] wow --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 29 +++++++++++++++++++ 1 file changed, 29 insertions(+) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index f1a8883d7b111..5399e8ebee056 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -5,6 +5,9 @@ package io.airbyte.cdk.load.data import com.fasterxml.jackson.databind.JsonNode +import io.github.oshai.kotlinlogging.KotlinLogging + +private val logger = KotlinLogging.logger {} sealed interface AirbyteType { /** @@ -67,6 +70,32 @@ data class UnionType( val options: Set, val isLegacyUnion: Boolean, ) : AirbyteType { + /** + * This is a hack to handle weird schemas like {type: [object, string]}. If a stream's top-level + * schema looks like this, we still want to be able to extract the object properties (i.e. treat + * it as though the string option didn't exist). + * + * @throws IllegalArgumentException if we cannot extract columns from this schema + */ + override fun asColumns(): LinkedHashMap { + logger.warn { "asColumns options=$options" } + val numObjectOptions = options.count { it.isObject } + if (numObjectOptions > 1) { + logger.error { "Can't extract columns from a schema with multiple object options" } + return LinkedHashMap() + } + + var retVal: LinkedHashMap + try { + retVal = options.first { it.isObject }.asColumns() + } catch (_: NoSuchElementException) { + logger.error { "Can't extract columns from a schema with no object options" } + retVal = LinkedHashMap() + } + logger.warn { "Union.asColumns retVal=$retVal" } + return retVal + } + companion object { fun of(options: Set, isLegacyUnion: Boolean = false): AirbyteType { if (options.size == 1) { From 68b82ae6e17f9db4319230a00b360a76e4b54db8 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 09:42:08 -0700 Subject: [PATCH 18/55] wire everything together --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 32 + .../cdk/load/orchestration/DbStreamLoader.kt | 11 - .../cdk/load/orchestration/DbWriter.kt | 15 - .../load/orchestration/DestinationHandler.kt | 6 +- .../DirectLoadTableStreamLoader.kt | 2 +- .../TypingDedupingStreamLoader.kt | 13 +- .../TypingDedupingWriter.kt | 7 + .../operation/BigQueryStorageOperationTest.kt | 213 ---- .../bigquery/BigQueryDestination.kt | 939 +++++++++--------- .../migrators/BigQueryDV2Migration.kt | 33 - .../BigQueryDirectLoadingStorageOperation.kt | 117 --- .../operation/BigQueryGcsStorageOperation.kt | 158 --- .../operation/BigQueryStorageOperation.kt | 177 ---- .../BigQueryDestinationHandler.kt | 28 +- .../typing_deduping/BigQuerySqlGenerator.kt | 17 +- ...igqueryDestinationInitialStatusGatherer.kt | 14 +- .../write/BigqueryInitialStateGatherer.kt | 20 - .../write/BigqueryRawTableOperations.kt | 103 +- .../bigquery/write/BigqueryWriter.kt | 13 +- .../BigqueryBatchStandardInsertLoader.kt | 17 +- .../destination/bigquery/BigqueryWriteTest.kt | 16 +- 21 files changed, 673 insertions(+), 1278 deletions(-) delete mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt delete mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperationTest.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDV2Migration.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryDirectLoadingStorageOperation.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryGcsStorageOperation.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperation.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index 5399e8ebee056..53280eabd00d1 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -5,6 +5,7 @@ package io.airbyte.cdk.load.data import com.fasterxml.jackson.databind.JsonNode +import io.airbyte.cdk.load.util.Jsons import io.github.oshai.kotlinlogging.KotlinLogging private val logger = KotlinLogging.logger {} @@ -96,6 +97,37 @@ data class UnionType( return retVal } + /** + * This matches legacy behavior. Some destinations handle legacy unions by choosing the "best" + * type from amongst the options. This is... not great, but it would be painful to change. + */ + fun chooseType(): AirbyteType { + check(isLegacyUnion) { "Cannot chooseType for a non-legacy union type" } + if (options.isEmpty()) { + return UnknownType(Jsons.createObjectNode()) + } + return options.minBy { + when (it) { + is ArrayType, + ArrayTypeWithoutSchema -> -2 + is ObjectType, + ObjectTypeWithEmptySchema, + ObjectTypeWithoutSchema -> -1 + StringType -> 0 + DateType -> 1 + TimeTypeWithoutTimezone -> 2 + TimeTypeWithTimezone -> 3 + TimestampTypeWithoutTimezone -> 4 + TimestampTypeWithTimezone -> 5 + NumberType -> 6 + IntegerType -> 7 + BooleanType -> 8 + is UnknownType -> 9 + is UnionType -> Int.MAX_VALUE + } + } + } + companion object { fun of(options: Set, isLegacyUnion: Boolean = false): AirbyteType { if (options.size == 1) { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt deleted file mode 100644 index 26a51492ecb08..0000000000000 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbStreamLoader.kt +++ /dev/null @@ -1,11 +0,0 @@ -package io.airbyte.cdk.load.orchestration - -import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.state.StreamProcessingFailed -import io.airbyte.cdk.load.write.StreamLoader - -class DbStreamLoader(override val stream: DestinationStream) : StreamLoader { - override suspend fun start() {} - - override suspend fun close(streamFailure: StreamProcessingFailed?) {} -} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt deleted file mode 100644 index 126e6224f7fcb..0000000000000 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DbWriter.kt +++ /dev/null @@ -1,15 +0,0 @@ -package io.airbyte.cdk.load.orchestration - -import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.write.DestinationWriter -import io.airbyte.cdk.load.write.StreamLoader - -class DbWriter : DestinationWriter { - override suspend fun setup() { - // create all namespaces - } - - override fun createStreamLoader(stream: DestinationStream): StreamLoader { - TODO("Not yet implemented") - } -} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt index 856b86a5a83f4..de966e5a70402 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration interface DestinationHandler { @@ -9,5 +13,5 @@ interface DestinationHandler { * This function should assume that all `namespaces` are valid identifiers, i.e. any special * characters have already been escaped, they respect identifier name length, etc. */ - fun createNamespaces(namespaces: List) + suspend fun createNamespaces(namespaces: List) } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt index e3104c1061141..fc779019caf8f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt @@ -19,7 +19,7 @@ class DirectLoadTableStreamLoader( // * truncate refresh setup } - override suspend fun close(streamFailure: StreamProcessingFailed?) { + override suspend fun close(hadNonzeroRecords: Boolean, streamFailure: StreamProcessingFailed?) { // TODO // * commit truncate refresh } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 02f004f3c4cdc..08804ecca00cf 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -302,7 +302,7 @@ class TypingDedupingStreamLoader( return NO_SUFFIX } - override suspend fun close(streamFailure: StreamProcessingFailed?) { + override suspend fun close(hadNonzeroRecords: Boolean, streamFailure: StreamProcessingFailed?) { val streamSuccessful = streamFailure == null // Overwrite the raw table before doing anything else. // This ensures that if T+D fails, we can easily retain the records on the next sync. @@ -331,26 +331,21 @@ class TypingDedupingStreamLoader( // Normal syncs should T+D regardless of status, so the user sees progress after every // attempt. // We know this is a normal sync, so initialRawTableStatus is nonnull. - if ( - !isTruncateSync && - syncSummary.recordsWritten == 0L && - !initialRawTableStatus.hasUnprocessedRecords - ) { + if (!isTruncateSync && !hadNonzeroRecords && !initialRawTableStatus.hasUnprocessedRecords) { logger.info { "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} because it had no records during this sync and no unprocessed records from a previous sync." } } else if ( isTruncateSync && (!streamSuccessful || - (syncSummary.recordsWritten == 0L && - !initialRawTableStatus.hasUnprocessedRecords)) + (!hadNonzeroRecords && !initialRawTableStatus.hasUnprocessedRecords)) ) { // But truncate syncs should only T+D if the sync was successful, since we're T+Ding // into a temp final table anyway. // We only run T+D if the current sync had some records, or a previous attempt wrote // some records to the temp raw table. logger.info { - "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} running as truncate sync. Stream success: $streamSuccessful; records written: ${syncSummary.recordsWritten}; temp raw table had records: ${initialRawTableStatus.hasUnprocessedRecords}" + "Skipping typing and deduping for stream ${stream.descriptor.toPrettyString()} running as truncate sync. Stream success: $streamSuccessful; had nonzero records: $hadNonzeroRecords; temp raw table had records: ${initialRawTableStatus.hasUnprocessedRecords}" } } else { // When targeting the temp final table, we want to read all the raw records diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index a4db18f1dc50a..46307cd8a29de 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -8,6 +8,7 @@ import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.ColumnNameGenerator import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.DestinationHandler import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.TableNameGenerator import io.airbyte.cdk.load.orchestration.TableNames @@ -25,6 +26,7 @@ class TypingDedupingWriter( private val finalTableColumnNameGenerator: ColumnNameGenerator, private val stateGatherer: DestinationInitialStatusGatherer, + private val destinationHandler: DestinationHandler, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, @@ -53,6 +55,11 @@ class TypingDedupingWriter( } Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> + destinationHandler.createNamespaces( + names.values.map { (tableNames, _) -> tableNames.rawTableName!!.namespace } + + names.values.map { (tableNames, _) -> tableNames.finalTableName!!.namespace } + ) + val initialInitialStatuses: Map = stateGatherer.gatherInitialStatus(names) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperationTest.kt deleted file mode 100644 index ef6ccf90b70b0..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperationTest.kt +++ /dev/null @@ -1,213 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.operation - -import com.fasterxml.jackson.databind.JsonNode -import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.BigQueryException -import com.google.cloud.bigquery.DatasetId -import com.google.cloud.bigquery.DatasetInfo -import com.google.cloud.bigquery.QueryJobConfiguration -import com.google.cloud.bigquery.TableResult -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage -import io.airbyte.commons.json.Jsons -import io.airbyte.commons.string.Strings -import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation.Companion.TMP_TABLE_SUFFIX -import io.airbyte.integrations.base.destination.typing_deduping.ImportType -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig -import io.airbyte.integrations.base.destination.typing_deduping.StreamId -import io.airbyte.integrations.destination.bigquery.BigQueryConsts -import io.airbyte.integrations.destination.bigquery.BigQueryDestination -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGeneratorIntegrationTest -import io.airbyte.protocol.models.v0.AirbyteMessage.Type -import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta -import java.nio.file.Files -import java.nio.file.Path -import java.util.Optional -import java.util.stream.Stream -import kotlin.test.assertEquals -import org.junit.jupiter.api.AfterEach -import org.junit.jupiter.api.BeforeEach -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.assertThrows -import org.junit.jupiter.api.parallel.Execution -import org.junit.jupiter.api.parallel.ExecutionMode - -/** - * Extremely barebones integration test for the direct inserts storage op. We should eventually: - * * Make something similar for the GCS storage op - * * Genericize this and put it in the CDK - * * Add assertions for all the columns, not just airbyte_data - * * Actually test all the methods on StorageOperation - */ -@Execution(ExecutionMode.CONCURRENT) -class BigQueryDirectLoadingStorageOperationTest { - private val randomString = Strings.addRandomSuffix("", "", 10) - private val streamId = - StreamId( - finalNamespace = "final_namespace_$randomString", - finalName = "final_name_$randomString", - rawNamespace = "raw_namespace_$randomString", - rawName = "raw_name_$randomString", - originalNamespace = "original_namespace_$randomString", - originalName = "original_name_$randomString", - ) - private val streamConfig = - StreamConfig( - streamId, - ImportType.APPEND, - emptyList(), - Optional.empty(), - LinkedHashMap(), - GENERATION_ID, - 0, - SYNC_ID, - ) - - @BeforeEach - fun setup() { - bq.create(DatasetInfo.of(streamId.rawNamespace)) - } - - @AfterEach - fun teardown() { - bq.delete( - DatasetId.of(streamId.rawNamespace), - BigQuery.DatasetDeleteOption.deleteContents() - ) - } - - @Test - fun testTransferStage() { - storageOperation.prepareStage(streamId, "") - storageOperation.prepareStage(streamId, TMP_TABLE_SUFFIX) - // Table is currently empty, so expect null generation. - assertEquals(null, storageOperation.getStageGeneration(streamId, TMP_TABLE_SUFFIX)) - - // Write one record to the real raw table - storageOperation.writeToStage( - streamConfig, - "", - Stream.of(record(1)), - ) - assertEquals( - listOf("""{"record_number": 1}"""), - // We write the raw data as a string column, not a JSON column, so use asText(). - dumpRawRecords("").map { it["_airbyte_data"].asText() }, - ) - - // And write one record to the temp final table - storageOperation.writeToStage( - streamConfig, - TMP_TABLE_SUFFIX, - Stream.of(record(2)), - ) - assertEquals( - listOf("""{"record_number": 2}"""), - dumpRawRecords(TMP_TABLE_SUFFIX).map { it["_airbyte_data"].asText() }, - ) - assertEquals(GENERATION_ID, storageOperation.getStageGeneration(streamId, TMP_TABLE_SUFFIX)) - - // If we transfer the records, we should end up with 2 records in the real raw table. - storageOperation.transferFromTempStage(streamId, TMP_TABLE_SUFFIX) - assertEquals( - listOf( - """{"record_number": 1}""", - """{"record_number": 2}""", - ), - dumpRawRecords("") - .sortedBy { - Jsons.deserialize(it["_airbyte_data"].asText())["record_number"].asLong() - } - .map { it["_airbyte_data"].asText() }, - ) - - // After transferring the records to the real table, the temp table should no longer exist. - assertEquals(404, assertThrows { dumpRawRecords(TMP_TABLE_SUFFIX) }.code) - } - - @Test - fun testOverwriteStage() { - // If we then create another temp raw table and _overwrite_ the real raw table, - // we should end up with a single raw record. - storageOperation.prepareStage(streamId, "") - storageOperation.prepareStage(streamId, TMP_TABLE_SUFFIX) - storageOperation.writeToStage( - streamConfig, - "", - Stream.of(record(3)), - ) - storageOperation.writeToStage( - streamConfig, - TMP_TABLE_SUFFIX, - Stream.of(record(4)), - ) - - storageOperation.overwriteStage(streamId, TMP_TABLE_SUFFIX) - - assertEquals( - listOf("""{"record_number": 4}"""), - dumpRawRecords("").map { it["_airbyte_data"].asText() }, - ) - assertEquals(404, assertThrows { dumpRawRecords(TMP_TABLE_SUFFIX) }.code) - } - - private fun dumpRawRecords(suffix: String): List { - val result: TableResult = - bq.query( - QueryJobConfiguration.of( - "SELECT * FROM " + streamId.rawTableId(BigQuerySqlGenerator.QUOTE, suffix) - ), - ) - return BigQuerySqlGeneratorIntegrationTest.toJsonRecords(result) - } - - private fun record(recordNumber: Int): PartialAirbyteMessage { - val serializedData = """{"record_number": $recordNumber}""" - return PartialAirbyteMessage() - .withType(Type.RECORD) - .withSerialized(serializedData) - .withRecord( - PartialAirbyteRecordMessage() - .withNamespace(streamId.originalNamespace) - .withStream(streamId.originalName) - .withEmittedAt(10_000) - .withMeta( - AirbyteRecordMessageMeta() - .withChanges(emptyList()) - .withAdditionalProperty( - JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY, - SYNC_ID, - ), - ) - .withData(Jsons.deserialize(serializedData)), - ) - } - - companion object { - private val config = - Jsons.deserialize(Files.readString(Path.of("secrets/credentials-gcs-staging.json"))) - private val bq = BigQueryDestination.getBigQuery(config) - private val projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText() - private val datasetLocation = config.get(BigQueryConsts.CONFIG_DATASET_LOCATION).asText() - private val storageOperation = - BigQueryDirectLoadingStorageOperation( - bq, - 15, - BigQueryRecordFormatter(), - BigQuerySqlGenerator(projectId, datasetLocation), - BigQueryDestinationHandler(bq, datasetLocation), - datasetLocation, - ) - - private const val SYNC_ID = 12L - private const val GENERATION_ID = 42L - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt index 4f90d8cccd870..bedbd4e414bb7 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt @@ -8,503 +8,476 @@ import com.google.auth.oauth2.GoogleCredentials import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.BigQueryException import com.google.cloud.bigquery.BigQueryOptions -import com.google.cloud.bigquery.QueryJobConfiguration -import com.google.cloud.storage.Storage -import com.google.cloud.storage.StorageOptions import io.airbyte.cdk.AirbyteDestinationRunner -import io.airbyte.cdk.integrations.BaseConnector -import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addAllStringsInConfigForDeinterpolation import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addThrowableForDeinterpolation -import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer -import io.airbyte.cdk.integrations.base.Destination -import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.* -import io.airbyte.cdk.integrations.base.SerializedAirbyteMessageConsumer -import io.airbyte.cdk.integrations.base.TypingAndDedupingFlag.getRawNamespaceOverride -import io.airbyte.cdk.integrations.destination.StreamSyncSummary -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage -import io.airbyte.cdk.integrations.destination.gcs.BaseGcsDestination -import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer -import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations -import io.airbyte.cdk.integrations.destination.operation.SyncOperation -import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat -import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations import io.airbyte.cdk.load.command.aws.AwsToolkitConstants -import io.airbyte.commons.exceptions.ConfigErrorException import io.airbyte.commons.json.Jsons.serialize -import io.airbyte.commons.json.Jsons.tryDeserialize -import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation -import io.airbyte.integrations.base.destination.operation.StandardStreamOperation -import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser -import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus -import io.airbyte.integrations.base.destination.typing_deduping.ImportType -import io.airbyte.integrations.base.destination.typing_deduping.InitialRawTableStatus -import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog -import io.airbyte.integrations.base.destination.typing_deduping.Sql -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig import io.airbyte.integrations.destination.bigquery.BigQueryConsts as bqConstants -import io.airbyte.integrations.destination.bigquery.BigQueryConsumerFactory.createDirectUploadConsumer -import io.airbyte.integrations.destination.bigquery.BigQueryConsumerFactory.createStagingConsumer -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDV2Migration -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState -import io.airbyte.integrations.destination.bigquery.migrators.BigqueryAirbyteMetaAndGenerationIdMigration -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation -import io.airbyte.integrations.destination.bigquery.operation.BigQueryGcsStorageOperation -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -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 -import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog import io.github.oshai.kotlinlogging.KotlinLogging import java.io.ByteArrayInputStream import java.io.IOException import java.nio.charset.StandardCharsets import java.util.* -import java.util.function.Consumer private val log = KotlinLogging.logger {} -class BigQueryDestination : BaseConnector(), Destination { - - override fun check(config: JsonNode): AirbyteConnectionStatus? { - - try { - - val datasetId = BigQueryUtils.getDatasetId(config) - val datasetLocation = BigQueryUtils.getDatasetLocation(config) - val bigquery = getBigQuery(config) - val uploadingMethod = BigQueryUtils.getLoadingMethod(config) - - val dataset = BigQueryUtils.getOrCreateDataset(bigquery, datasetId, datasetLocation) - if (dataset.location != datasetLocation) { - throw ConfigErrorException( - "Actual dataset location doesn't match to location from config", - ) - } - - val queryConfig = - QueryJobConfiguration.newBuilder( - String.format( - "SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES` LIMIT 1;", - datasetId, - ), - ) - .setUseLegacySql(false) - .build() - - val result = BigQueryUtils.executeQuery(bigquery, queryConfig) - - if (result.getLeft() == null) { - - return AirbyteConnectionStatus() - .withStatus(AirbyteConnectionStatus.Status.FAILED) - .withMessage(result.right) - } - - if (UploadingMethod.GCS == uploadingMethod) { - - val gcsStatus = checkGcsAccessPermission(config) - if (gcsStatus!!.status != AirbyteConnectionStatus.Status.SUCCEEDED) { - return gcsStatus - } - - // Copy a temporary dataset to confirm copy permissions are working - val bigQueryStatus = checkBigQueryCopyPermission(config) - - if (bigQueryStatus.status != AirbyteConnectionStatus.Status.SUCCEEDED) { - return bigQueryStatus - } - } - - return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) - } catch (e: Exception) { - log.error(e) { "Check failed." } - throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) - } - } - - /** - * This method performs a copy operation to copy data into a temporary table on BigQuery to - * check if the existing permissions are sufficient to copy data. If the permissions are not - * sufficient, then an exception is thrown with a message showing the missing permission - */ - private fun checkBigQueryCopyPermission(config: JsonNode): AirbyteConnectionStatus { - - // TODO: Need to add a step in this method to first check permissions - // using testIamPermissions before trying the actual copying of data - // Created issue for tracking: - // https://github.com/airbytehq/airbyte-internal-issues/issues/8888 - - val datasetLocation = BigQueryUtils.getDatasetLocation(config) - val bigquery = getBigQuery(config) - - val gcsNameTransformer = GcsNameTransformer() - val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) - val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) - val gcsOperations = - GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) - - val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() - val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) - val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) - val defaultDataset = BigQueryUtils.getDatasetId(config) - - val finalTableName = - "_airbyte_bigquery_connection_test_" + - UUID.randomUUID().toString().replace("-".toRegex(), "") - - val rawDatasetOverride: String = - if (getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).isPresent) { - getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).get() - } else { - JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE - } - - val streamId = - sqlGenerator.buildStreamId(defaultDataset, finalTableName, rawDatasetOverride) - - try { - - // Copy a dataset into a BigQuery table to confirm the copy operation is working - // correctly with the existing permissions - - val streamConfig = - StreamConfig( - id = streamId, - postImportAction = ImportType.APPEND, - primaryKey = listOf(), - cursor = Optional.empty(), - columns = linkedMapOf(), - generationId = 1, - minimumGenerationId = 1, - syncId = 0 - ) - - // None of the fields in destination initial status matter - // for a dummy sync with type-dedupe disabled. We only look at these - // when we perform final table related setup operations. - // We just need the streamId to perform the calls in streamOperation. - - val initialStatus = - DestinationInitialStatus( - streamConfig = streamConfig, - isFinalTablePresent = false, - initialRawTableStatus = - InitialRawTableStatus( - rawTableExists = false, - hasUnprocessedRecords = true, - maxProcessedTimestamp = Optional.empty() - ), - initialTempRawTableStatus = - InitialRawTableStatus( - rawTableExists = false, - hasUnprocessedRecords = true, - maxProcessedTimestamp = Optional.empty() - ), - isSchemaMismatch = true, - isFinalTableEmpty = true, - destinationState = BigQueryDestinationState(needsSoftReset = false), - finalTempTableGenerationId = null, - finalTableGenerationId = null, - ) - - // We simulate a mini-sync to see the raw table code path is exercised. and disable T+D - destinationHandler.createNamespaces(setOf(defaultDataset, rawDatasetOverride)) - - val bigQueryGcsStorageOperations = - BigQueryGcsStorageOperation( - gcsOperations, - gcsConfig, - gcsNameTransformer, - keepStagingFiles, - bigquery, - sqlGenerator, - destinationHandler, - ) - - val streamOperation: StagingStreamOperations = - StagingStreamOperations( - bigQueryGcsStorageOperations, - initialStatus, - FileUploadFormat.CSV, - V2_WITH_GENERATION, - disableTypeDedupe = true - ) - - // Dummy message - val data = - """ - {"testKey": "testValue"} - """.trimIndent() - - val message = - PartialAirbyteMessage() - .withSerialized(data) - .withRecord( - PartialAirbyteRecordMessage() - .withEmittedAt(System.currentTimeMillis()) - .withMeta( - AirbyteRecordMessageMeta(), - ), - ) - - streamOperation.writeRecords(streamConfig, listOf(message).stream()) - streamOperation.finalizeTable( - streamConfig, - StreamSyncSummary(1, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE), - ) - - // Note: Deletion of the temporary table is being done in the finally block below - - return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) - } catch (e: Exception) { - - log.error(e) { "checkGcsCopyPermission failed." } - - throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) - } finally { - - try { - // In the finally block, clean up the raw table - // If there was an exception in the flow above, then the table may still exist - destinationHandler.execute( - Sql.of( - "DROP TABLE IF EXISTS $projectId.${streamId.rawNamespace}.${streamId.rawName};", - ), - ) - } catch (e: Exception) { - log.error(e) { - "Error while cleaning up the temporary table... not throwing a new exception" - } - } - } - } - - /** - * This method does two checks: 1) permissions related to the bucket, and 2) the ability to - * create and delete an actual file. The latter is important because even if the service account - * may have the proper permissions, the HMAC keys can only be verified by running the actual GCS - * check. - */ - private fun checkGcsAccessPermission(config: JsonNode): AirbyteConnectionStatus? { - val loadingMethod = config[bqConstants.LOADING_METHOD] - val bucketName = loadingMethod[bqConstants.GCS_BUCKET_NAME].asText() - val missingPermissions: MutableList = ArrayList() - - try { - val credentials = getServiceAccountCredentials(config) - val storage: Storage = - StorageOptions.newBuilder() - .setProjectId(config[bqConstants.CONFIG_PROJECT_ID].asText()) - .setCredentials(credentials) - .setHeaderProvider(BigQueryUtils.headerProvider) - .build() - .service - val permissionsCheckStatusList: List = - storage.testIamPermissions(bucketName, REQUIRED_GCS_PERMISSIONS) - - // testIamPermissions returns a list of booleans - // in the same order of the presented permissions list - missingPermissions.addAll( - permissionsCheckStatusList - .asSequence() - .withIndex() - .filter { !it.value } - .map { REQUIRED_GCS_PERMISSIONS[it.index] } - .toList(), - ) - - val gcsDestination: BaseGcsDestination = object : BaseGcsDestination() {} - val gcsJsonNodeConfig = BigQueryUtils.getGcsJsonNodeConfig(config) - return gcsDestination.check(gcsJsonNodeConfig) - } catch (e: Exception) { - val message = StringBuilder("Cannot access the GCS bucket.") - if (!missingPermissions.isEmpty()) { - message - .append(" The following permissions are missing on the service account: ") - .append(java.lang.String.join(", ", missingPermissions)) - .append(".") - } - message.append( - " Please make sure the service account can access the bucket path, and the HMAC keys are correct.", - ) - - log.error(e) { message.toString() } - throw ConfigErrorException( - "Could not access the GCS bucket with the provided configuration.\n", - e, - ) - } - } - - /** - * Returns a [AirbyteMessageConsumer] based on whether the uploading mode is STANDARD INSERTS or - * using STAGING - * - * @param config - * - integration-specific configuration object as json. e.g. { "username": "airbyte", - * "password": "super secure" } - * @param catalog - * - schema of the incoming messages. - */ - override fun getConsumer( - config: JsonNode, - catalog: ConfiguredAirbyteCatalog, - outputRecordCollector: Consumer - ): AirbyteMessageConsumer? { - throw UnsupportedOperationException("Should use getSerializedMessageConsumer") - } - - @Throws(Exception::class) - override fun getSerializedMessageConsumer( - config: JsonNode, - catalog: ConfiguredAirbyteCatalog, - outputRecordCollector: Consumer - ): SerializedAirbyteMessageConsumer { - val uploadingMethod = BigQueryUtils.getLoadingMethod(config) - val defaultNamespace = BigQueryUtils.getDatasetId(config) - val disableTypeDedupe = BigQueryUtils.getDisableTypeDedupFlag(config) - val datasetLocation = BigQueryUtils.getDatasetLocation(config) - val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() - val bigquery = getBigQuery(config) - val rawNamespaceOverride = getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET) - - addAllStringsInConfigForDeinterpolation(config) - val serviceAccountKey = config[bqConstants.CONFIG_CREDS] - if (serviceAccountKey != null) { - // If the service account key is a non-null string, we will try to - // deserialize it. Otherwise, we will let the Google library find it in - // the environment during the client initialization. - if (serviceAccountKey.isTextual) { - // There are cases where we fail to deserialize the service account key. In these - // cases, we - // shouldn't do anything. - // Google's creds library is more lenient with JSON-parsing than Jackson, and I'd - // rather just let it - // go. - tryDeserialize(serviceAccountKey.asText()).ifPresent { obj: JsonNode -> - addAllStringsInConfigForDeinterpolation(obj) - } - } else { - addAllStringsInConfigForDeinterpolation(serviceAccountKey) - } - } - - val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) - val parsedCatalog = - parseCatalog( - sqlGenerator, - defaultNamespace, - rawNamespaceOverride.orElse(JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE), - catalog, - ) - val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) - - val migrations = - listOf( - BigQueryDV2Migration(sqlGenerator, bigquery), - BigqueryAirbyteMetaAndGenerationIdMigration(bigquery), - ) - - if (uploadingMethod == UploadingMethod.STANDARD) { - val bigQueryClientChunkSize = BigQueryUtils.getBigQueryClientChunkSize(config) - val bigQueryLoadingStorageOperation = - BigQueryDirectLoadingStorageOperation( - bigquery, - bigQueryClientChunkSize, - BigQueryRecordFormatter(), - sqlGenerator, - destinationHandler, - datasetLocation, - ) - val syncOperation = - DefaultSyncOperation( - parsedCatalog, - destinationHandler, - defaultNamespace, - { initialStatus: DestinationInitialStatus, disableTD - -> - StandardStreamOperation( - bigQueryLoadingStorageOperation, - initialStatus, - disableTD - ) - }, - migrations, - disableTypeDedupe, - ) - return createDirectUploadConsumer( - outputRecordCollector, - syncOperation, - catalog, - defaultNamespace, - ) - } - - val gcsNameTransformer = GcsNameTransformer() - val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) - val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) - val gcsOperations = - GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) - - val bigQueryGcsStorageOperations = - BigQueryGcsStorageOperation( - gcsOperations, - gcsConfig, - gcsNameTransformer, - keepStagingFiles, - bigquery, - sqlGenerator, - destinationHandler, - ) - val syncOperation: SyncOperation = - DefaultSyncOperation( - parsedCatalog, - destinationHandler, - defaultNamespace, - { initialStatus: DestinationInitialStatus, disableTD -> - StagingStreamOperations( - bigQueryGcsStorageOperations, - initialStatus, - FileUploadFormat.CSV, - V2_WITH_GENERATION, - disableTD - ) - }, - migrations, - disableTypeDedupe, - ) - return createStagingConsumer( - outputRecordCollector, - syncOperation, - catalog, - defaultNamespace, - ) - } - - private fun parseCatalog( - sqlGenerator: BigQuerySqlGenerator, - defaultNamespace: String, - rawNamespaceOverride: String, - catalog: ConfiguredAirbyteCatalog - ): ParsedCatalog { - val catalogParser = - CatalogParser( - sqlGenerator, - defaultNamespace = defaultNamespace, - rawNamespace = rawNamespaceOverride, - ) - - return catalogParser.parseCatalog(catalog) - } - - override val isV2Destination: Boolean - get() = true - +class BigQueryDestination /*: BaseConnector(), Destination */ { + // + // override fun check(config: JsonNode): AirbyteConnectionStatus? { + // + // try { + // + // val datasetId = BigQueryUtils.getDatasetId(config) + // val datasetLocation = BigQueryUtils.getDatasetLocation(config) + // val bigquery = getBigQuery(config) + // val uploadingMethod = BigQueryUtils.getLoadingMethod(config) + // + // val dataset = BigQueryUtils.getOrCreateDataset(bigquery, datasetId, + // datasetLocation) + // if (dataset.location != datasetLocation) { + // throw ConfigErrorException( + // "Actual dataset location doesn't match to location from config", + // ) + // } + // + // val queryConfig = + // QueryJobConfiguration.newBuilder( + // String.format( + // "SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES` LIMIT 1;", + // datasetId, + // ), + // ) + // .setUseLegacySql(false) + // .build() + // + // val result = BigQueryUtils.executeQuery(bigquery, queryConfig) + // + // if (result.getLeft() == null) { + // + // return AirbyteConnectionStatus() + // .withStatus(AirbyteConnectionStatus.Status.FAILED) + // .withMessage(result.right) + // } + // + // if (UploadingMethod.GCS == uploadingMethod) { + // + // val gcsStatus = checkGcsAccessPermission(config) + // if (gcsStatus!!.status != AirbyteConnectionStatus.Status.SUCCEEDED) { + // return gcsStatus + // } + // + // // Copy a temporary dataset to confirm copy permissions are working + // val bigQueryStatus = checkBigQueryCopyPermission(config) + // + // if (bigQueryStatus.status != AirbyteConnectionStatus.Status.SUCCEEDED) { + // return bigQueryStatus + // } + // } + // + // return + // AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) + // } catch (e: Exception) { + // log.error(e) { "Check failed." } + // throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) + // } + // } + // + // /** + // * This method performs a copy operation to copy data into a temporary table on BigQuery + // to + // * check if the existing permissions are sufficient to copy data. If the permissions are + // not + // * sufficient, then an exception is thrown with a message showing the missing permission + // */ + // private fun checkBigQueryCopyPermission(config: JsonNode): AirbyteConnectionStatus { + // + // // TODO: Need to add a step in this method to first check permissions + // // using testIamPermissions before trying the actual copying of data + // // Created issue for tracking: + // // https://github.com/airbytehq/airbyte-internal-issues/issues/8888 + // + // val datasetLocation = BigQueryUtils.getDatasetLocation(config) + // val bigquery = getBigQuery(config) + // + // val gcsNameTransformer = GcsNameTransformer() + // val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) + // val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) + // val gcsOperations = + // GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) + // + // val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() + // val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) + // val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) + // val defaultDataset = BigQueryUtils.getDatasetId(config) + // + // val finalTableName = + // "_airbyte_bigquery_connection_test_" + + // UUID.randomUUID().toString().replace("-".toRegex(), "") + // + // val rawDatasetOverride: String = + // if (getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).isPresent) { + // getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).get() + // } else { + // JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE + // } + // + // val streamId = + // sqlGenerator.buildStreamId(defaultDataset, finalTableName, rawDatasetOverride) + // + // try { + // + // // Copy a dataset into a BigQuery table to confirm the copy operation is working + // // correctly with the existing permissions + // + // val streamConfig = + // StreamConfig( + // id = streamId, + // postImportAction = ImportType.APPEND, + // primaryKey = listOf(), + // cursor = Optional.empty(), + // columns = linkedMapOf(), + // generationId = 1, + // minimumGenerationId = 1, + // syncId = 0 + // ) + // + // // None of the fields in destination initial status matter + // // for a dummy sync with type-dedupe disabled. We only look at these + // // when we perform final table related setup operations. + // // We just need the streamId to perform the calls in streamOperation. + // + // val initialStatus = + // DestinationInitialStatus( + // streamConfig = streamConfig, + // isFinalTablePresent = false, + // initialRawTableStatus = + // InitialRawTableStatus( + // rawTableExists = false, + // hasUnprocessedRecords = true, + // maxProcessedTimestamp = Optional.empty() + // ), + // initialTempRawTableStatus = + // InitialRawTableStatus( + // rawTableExists = false, + // hasUnprocessedRecords = true, + // maxProcessedTimestamp = Optional.empty() + // ), + // isSchemaMismatch = true, + // isFinalTableEmpty = true, + // destinationState = BigQueryDestinationState(needsSoftReset = false), + // finalTempTableGenerationId = null, + // finalTableGenerationId = null, + // ) + // + // // We simulate a mini-sync to see the raw table code path is exercised. and + // disable T+D + // destinationHandler.createNamespaces(setOf(defaultDataset, rawDatasetOverride)) + // + // val bigQueryGcsStorageOperations = + // BigQueryGcsStorageOperation( + // gcsOperations, + // gcsConfig, + // gcsNameTransformer, + // keepStagingFiles, + // bigquery, + // sqlGenerator, + // destinationHandler, + // ) + // + // val streamOperation: StagingStreamOperations = + // StagingStreamOperations( + // bigQueryGcsStorageOperations, + // initialStatus, + // FileUploadFormat.CSV, + // V2_WITH_GENERATION, + // disableTypeDedupe = true + // ) + // + // // Dummy message + // val data = + // """ + // {"testKey": "testValue"} + // """.trimIndent() + // + // val message = + // PartialAirbyteMessage() + // .withSerialized(data) + // .withRecord( + // PartialAirbyteRecordMessage() + // .withEmittedAt(System.currentTimeMillis()) + // .withMeta( + // AirbyteRecordMessageMeta(), + // ), + // ) + // + // streamOperation.writeRecords(streamConfig, listOf(message).stream()) + // streamOperation.finalizeTable( + // streamConfig, + // StreamSyncSummary(1, + // AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE), + // ) + // + // // Note: Deletion of the temporary table is being done in the finally block below + // + // return + // AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) + // } catch (e: Exception) { + // + // log.error(e) { "checkGcsCopyPermission failed." } + // + // throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) + // } finally { + // + // try { + // // In the finally block, clean up the raw table + // // If there was an exception in the flow above, then the table may still exist + // destinationHandler.execute( + // Sql.of( + // "DROP TABLE IF EXISTS + // $projectId.${streamId.rawNamespace}.${streamId.rawName};", + // ), + // ) + // } catch (e: Exception) { + // log.error(e) { + // "Error while cleaning up the temporary table... not throwing a new + // exception" + // } + // } + // } + // } + // + // /** + // * This method does two checks: 1) permissions related to the bucket, and 2) the ability + // to + // * create and delete an actual file. The latter is important because even if the service + // account + // * may have the proper permissions, the HMAC keys can only be verified by running the + // actual GCS + // * check. + // */ + // private fun checkGcsAccessPermission(config: JsonNode): AirbyteConnectionStatus? { + // val loadingMethod = config[bqConstants.LOADING_METHOD] + // val bucketName = loadingMethod[bqConstants.GCS_BUCKET_NAME].asText() + // val missingPermissions: MutableList = ArrayList() + // + // try { + // val credentials = getServiceAccountCredentials(config) + // val storage: Storage = + // StorageOptions.newBuilder() + // .setProjectId(config[bqConstants.CONFIG_PROJECT_ID].asText()) + // .setCredentials(credentials) + // .setHeaderProvider(BigQueryUtils.headerProvider) + // .build() + // .service + // val permissionsCheckStatusList: List = + // storage.testIamPermissions(bucketName, REQUIRED_GCS_PERMISSIONS) + // + // // testIamPermissions returns a list of booleans + // // in the same order of the presented permissions list + // missingPermissions.addAll( + // permissionsCheckStatusList + // .asSequence() + // .withIndex() + // .filter { !it.value } + // .map { REQUIRED_GCS_PERMISSIONS[it.index] } + // .toList(), + // ) + // + // val gcsDestination: BaseGcsDestination = object : BaseGcsDestination() {} + // val gcsJsonNodeConfig = BigQueryUtils.getGcsJsonNodeConfig(config) + // return gcsDestination.check(gcsJsonNodeConfig) + // } catch (e: Exception) { + // val message = StringBuilder("Cannot access the GCS bucket.") + // if (!missingPermissions.isEmpty()) { + // message + // .append(" The following permissions are missing on the service account: ") + // .append(java.lang.String.join(", ", missingPermissions)) + // .append(".") + // } + // message.append( + // " Please make sure the service account can access the bucket path, and the + // HMAC keys are correct.", + // ) + // + // log.error(e) { message.toString() } + // throw ConfigErrorException( + // "Could not access the GCS bucket with the provided configuration.\n", + // e, + // ) + // } + // } + // + // /** + // * Returns a [AirbyteMessageConsumer] based on whether the uploading mode is STANDARD + // INSERTS or + // * using STAGING + // * + // * @param config + // * - integration-specific configuration object as json. e.g. { "username": "airbyte", + // * "password": "super secure" } + // * @param catalog + // * - schema of the incoming messages. + // */ + // override fun getConsumer( + // config: JsonNode, + // catalog: ConfiguredAirbyteCatalog, + // outputRecordCollector: Consumer + // ): AirbyteMessageConsumer? { + // throw UnsupportedOperationException("Should use getSerializedMessageConsumer") + // } + // + // @Throws(Exception::class) + // override fun getSerializedMessageConsumer( + // config: JsonNode, + // catalog: ConfiguredAirbyteCatalog, + // outputRecordCollector: Consumer + // ): SerializedAirbyteMessageConsumer { + // val uploadingMethod = BigQueryUtils.getLoadingMethod(config) + // val defaultNamespace = BigQueryUtils.getDatasetId(config) + // val disableTypeDedupe = BigQueryUtils.getDisableTypeDedupFlag(config) + // val datasetLocation = BigQueryUtils.getDatasetLocation(config) + // val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() + // val bigquery = getBigQuery(config) + // val rawNamespaceOverride = getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET) + // + // addAllStringsInConfigForDeinterpolation(config) + // val serviceAccountKey = config[bqConstants.CONFIG_CREDS] + // if (serviceAccountKey != null) { + // // If the service account key is a non-null string, we will try to + // // deserialize it. Otherwise, we will let the Google library find it in + // // the environment during the client initialization. + // if (serviceAccountKey.isTextual) { + // // There are cases where we fail to deserialize the service account key. In + // these + // // cases, we + // // shouldn't do anything. + // // Google's creds library is more lenient with JSON-parsing than Jackson, and + // I'd + // // rather just let it + // // go. + // tryDeserialize(serviceAccountKey.asText()).ifPresent { obj: JsonNode -> + // addAllStringsInConfigForDeinterpolation(obj) + // } + // } else { + // addAllStringsInConfigForDeinterpolation(serviceAccountKey) + // } + // } + // + // val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) + // val parsedCatalog = + // parseCatalog( + // sqlGenerator, + // defaultNamespace, + // + // rawNamespaceOverride.orElse(JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE), + // catalog, + // ) + // val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) + // + // val migrations = + // listOf( + // BigQueryDV2Migration(sqlGenerator, bigquery), + // BigqueryAirbyteMetaAndGenerationIdMigration(bigquery), + // ) + // + // if (uploadingMethod == UploadingMethod.STANDARD) { + // val bigQueryClientChunkSize = BigQueryUtils.getBigQueryClientChunkSize(config) + // val bigQueryLoadingStorageOperation = + // BigQueryDirectLoadingStorageOperation( + // bigquery, + // bigQueryClientChunkSize, + // BigQueryRecordFormatter(), + // sqlGenerator, + // destinationHandler, + // datasetLocation, + // ) + // val syncOperation = + // DefaultSyncOperation( + // parsedCatalog, + // destinationHandler, + // defaultNamespace, + // { initialStatus: DestinationInitialStatus, + // disableTD + // -> + // StandardStreamOperation( + // bigQueryLoadingStorageOperation, + // initialStatus, + // disableTD + // ) + // }, + // migrations, + // disableTypeDedupe, + // ) + // return createDirectUploadConsumer( + // outputRecordCollector, + // syncOperation, + // catalog, + // defaultNamespace, + // ) + // } + // + // val gcsNameTransformer = GcsNameTransformer() + // val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) + // val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) + // val gcsOperations = + // GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) + // + // val bigQueryGcsStorageOperations = + // BigQueryGcsStorageOperation( + // gcsOperations, + // gcsConfig, + // gcsNameTransformer, + // keepStagingFiles, + // bigquery, + // sqlGenerator, + // destinationHandler, + // ) + // val syncOperation: SyncOperation = + // DefaultSyncOperation( + // parsedCatalog, + // destinationHandler, + // defaultNamespace, + // { initialStatus: DestinationInitialStatus, disableTD + // -> + // StagingStreamOperations( + // bigQueryGcsStorageOperations, + // initialStatus, + // FileUploadFormat.CSV, + // V2_WITH_GENERATION, + // disableTD + // ) + // }, + // migrations, + // disableTypeDedupe, + // ) + // return createStagingConsumer( + // outputRecordCollector, + // syncOperation, + // catalog, + // defaultNamespace, + // ) + // } + // + // private fun parseCatalog( + // sqlGenerator: BigQuerySqlGenerator, + // defaultNamespace: String, + // rawNamespaceOverride: String, + // catalog: ConfiguredAirbyteCatalog + // ): ParsedCatalog { + // val catalogParser = + // CatalogParser( + // sqlGenerator, + // defaultNamespace = defaultNamespace, + // rawNamespace = rawNamespaceOverride, + // ) + // + // return catalogParser.parseCatalog(catalog) + // } + // + // override val isV2Destination: Boolean + // get() = true + // companion object { private val REQUIRED_GCS_PERMISSIONS = diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDV2Migration.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDV2Migration.kt deleted file mode 100644 index 6f23c33f5672d..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDV2Migration.kt +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.migrators - -import com.google.cloud.bigquery.BigQuery -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.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryV1V2Migrator -import io.github.oshai.kotlinlogging.KotlinLogging -import kotlin.math.log - -class BigQueryDV2Migration(private val sqlGenerator: BigQuerySqlGenerator, bigQuery: BigQuery) : - Migration { - private val log = KotlinLogging.logger {} - private val legacyV1V2migrator = BigQueryV1V2Migrator(bigQuery, BigQuerySQLNameTransformer()) - override fun migrateIfNecessary( - destinationHandler: DestinationHandler, - stream: StreamConfig, - state: DestinationInitialStatus - ): Migration.MigrationResult { - log.info { "Initializing DV2 Migration check" } - legacyV1V2migrator.migrateIfNecessary(sqlGenerator, destinationHandler, stream) - // Invalidate state because rawTableExists could be false but we don't use it yet for - // anything ? - return Migration.MigrationResult(BigQueryDestinationState(false), true) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryDirectLoadingStorageOperation.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryDirectLoadingStorageOperation.kt deleted file mode 100644 index 9ebca355c58e6..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryDirectLoadingStorageOperation.kt +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.operation - -import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.BigQueryException -import com.google.cloud.bigquery.FormatOptions -import com.google.cloud.bigquery.JobId -import com.google.cloud.bigquery.JobInfo -import com.google.cloud.bigquery.TableDataWriteChannel -import com.google.cloud.bigquery.TableId -import com.google.cloud.bigquery.WriteChannelConfiguration -import com.google.common.util.concurrent.RateLimiter -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage -import io.airbyte.commons.exceptions.ConfigErrorException -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig -import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.github.oshai.kotlinlogging.KotlinLogging -import java.nio.ByteBuffer -import java.nio.charset.StandardCharsets -import java.util.stream.Stream - -private val log = KotlinLogging.logger {} - -class BigQueryDirectLoadingStorageOperation( - bigquery: BigQuery, - private val bigQueryClientChunkSize: Int?, - private val bigQueryRecordFormatter: BigQueryRecordFormatter, - sqlGenerator: BigQuerySqlGenerator, - destinationHandler: BigQueryDestinationHandler, - datasetLocation: String -) : - BigQueryStorageOperation>( - bigquery, - sqlGenerator, - destinationHandler, - datasetLocation, - ) { - private val rateLimiter: RateLimiter = RateLimiter.create(0.07) - companion object { - const val HTTP_STATUS_CODE_FORBIDDEN = 403 - const val HTTP_STATUS_CODE_NOT_FOUND = 404 - - val CONFIG_ERROR_MSG = - """ - |Failed to write to destination schema. - | 1. Make sure you have all required permissions for writing to the schema. - | 2. Make sure that the actual destination schema's location corresponds to location provided in connector's config. - | 3. Try to change the "Destination schema" from "Mirror Source Structure" (if it's set) tp the "Destination Default" option. - |More details: - |""".trimMargin() - } - override fun writeToStage( - streamConfig: StreamConfig, - suffix: String, - data: Stream - ) { - // TODO: why do we need ratelimiter, and using unstable API from Google's guava - rateLimiter.acquire() - val tableId = tableId(streamConfig.id, suffix) - log.info { - "Writing data to table $tableId with schema ${BigQueryRecordFormatter.SCHEMA_V2}" - } - val writeChannel = initWriteChannel(tableId) - writeChannel.use { - data.forEach { record -> - val byteArray = - "${bigQueryRecordFormatter.formatRecord(record, streamConfig.generationId)} ${System.lineSeparator()}".toByteArray( - StandardCharsets.UTF_8, - ) - it.write(ByteBuffer.wrap(byteArray)) - } - } - log.info { "Writing to channel completed for $tableId" } - val job = writeChannel.job - BigQueryUtils.waitForJobFinish(job) - } - - private fun initWriteChannel(tableId: TableId): TableDataWriteChannel { - val writeChannelConfiguration = - WriteChannelConfiguration.newBuilder(tableId) - .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) - .setSchema(BigQueryRecordFormatter.SCHEMA_V2) - .setFormatOptions(FormatOptions.json()) - .build() // new-line delimited json. - - val job = - JobId.newBuilder() - .setRandomJob() - .setLocation(datasetLocation) - .setProject(bigquery.options.projectId) - .build() - - val writer: TableDataWriteChannel - - try { - writer = bigquery.writer(job, writeChannelConfiguration) - } catch (e: BigQueryException) { - if (e.code == HTTP_STATUS_CODE_FORBIDDEN || e.code == HTTP_STATUS_CODE_NOT_FOUND) { - throw ConfigErrorException(CONFIG_ERROR_MSG + e) - } else { - throw BigQueryException(e.code, e.message) - } - } - - // this this optional value. If not set - use default client's value (15MiG) - if (bigQueryClientChunkSize != null) { - writer.setChunkSize(bigQueryClientChunkSize) - } - return writer - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryGcsStorageOperation.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryGcsStorageOperation.kt deleted file mode 100644 index 35dde19cb9b7f..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryGcsStorageOperation.kt +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.operation - -import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.BigQueryException -import com.google.cloud.bigquery.FormatOptions -import com.google.cloud.bigquery.Job -import com.google.cloud.bigquery.JobInfo -import com.google.cloud.bigquery.LoadJobConfiguration -import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig -import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer -import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations -import io.airbyte.cdk.integrations.destination.record_buffer.SerializableBuffer -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig -import io.airbyte.integrations.base.destination.typing_deduping.StreamId -import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.github.oshai.kotlinlogging.KotlinLogging -import java.util.* -import org.joda.time.DateTime -import org.joda.time.DateTimeZone - -private val log = KotlinLogging.logger {} - -class BigQueryGcsStorageOperation( - private val gcsStorageOperations: GcsStorageOperations, - private val gcsConfig: GcsDestinationConfig, - private val gcsNameTransformer: GcsNameTransformer, - private val keepStagingFiles: Boolean, - bigquery: BigQuery, - sqlGenerator: BigQuerySqlGenerator, - destinationHandler: BigQueryDestinationHandler -) : - BigQueryStorageOperation( - bigquery, - sqlGenerator, - destinationHandler, - datasetLocation = gcsConfig.bucketRegion!! - ) { - private val connectionId = UUID.randomUUID() - private val syncDateTime = DateTime.now(DateTimeZone.UTC) - override fun prepareStage(streamId: StreamId, suffix: String, replace: Boolean) { - super.prepareStage(streamId, suffix, replace) - // prepare staging bucket - // TODO should this also use the suffix? - log.info { "Creating bucket ${gcsConfig.bucketName}" } - gcsStorageOperations.createBucketIfNotExists() - } - - override fun cleanupStage(streamId: StreamId) { - if (keepStagingFiles) return - - val stagingRootPath = stagingRootPath(streamId) - log.info { "Cleaning up staging path at $stagingRootPath" } - gcsStorageOperations.dropBucketObject(stagingRootPath) - } - - override fun writeToStage( - streamConfig: StreamConfig, - suffix: String, - data: SerializableBuffer - ) { - val stagedFileName: String = - uploadRecordsToStage(streamConfig.id, suffix, data, streamConfig.generationId) - copyIntoTableFromStage(streamConfig.id, suffix, stagedFileName) - } - - private fun uploadRecordsToStage( - streamId: StreamId, - suffix: String, - buffer: SerializableBuffer, - generationId: Long, - ): String { - val objectPath: String = stagingFullPath(streamId) - log.info { - "Uploading records to for ${streamId.rawNamespace}.${streamId.rawName}$suffix to path $objectPath" - } - return gcsStorageOperations.uploadRecordsToBucket( - buffer, - streamId.rawNamespace, - objectPath, - generationId - ) - } - - private fun copyIntoTableFromStage(streamId: StreamId, suffix: String, stagedFileName: String) { - val tableId = tableId(streamId, suffix) - val stagingPath = stagingFullPath(streamId) - val fullFilePath = "gs://${gcsConfig.bucketName}/$stagingPath$stagedFileName" - log.info { "Uploading records from file $fullFilePath to target Table $tableId" } - val configuration = - LoadJobConfiguration.builder(tableId, fullFilePath) - .setFormatOptions(FormatOptions.csv()) - .setSchema(BigQueryRecordFormatter.SCHEMA_V2) - .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND) - .setJobTimeoutMs(600000L) // 10 min - .build() - - val loadJob: Job = this.bigquery.create(JobInfo.of(configuration)) - log.info { - "[${loadJob.jobId}] Created a new job to upload record(s) to target table $tableId: $loadJob" - } - try { - BigQueryUtils.waitForJobFinish(loadJob) - log.info { - "[${loadJob.jobId}] Target table $tableId is successfully appended with staging files" - } - } catch (e: BigQueryException) { - throw RuntimeException( - String.format( - "[%s] Failed to upload staging files to destination table %s", - loadJob.jobId, - tableId - ), - e - ) - } catch (e: InterruptedException) { - throw RuntimeException( - String.format( - "[%s] Failed to upload staging files to destination table %s", - loadJob.jobId, - tableId - ), - e - ) - } - } - - private fun stagingFullPath(streamId: StreamId): String { - return gcsNameTransformer.applyDefaultCase( - String.format( - "%s%s/%02d/%02d/%02d/%s/", - stagingRootPath(streamId), - syncDateTime.year().get(), - syncDateTime.monthOfYear().get(), - syncDateTime.dayOfMonth().get(), - syncDateTime.hourOfDay().get(), - connectionId - ) - ) - } - - private fun stagingRootPath(streamId: StreamId): String { - return gcsNameTransformer.applyDefaultCase( - String.format( - "%s/%s_%s/", - gcsConfig.bucketPath, - gcsNameTransformer.convertStreamName(streamId.rawNamespace), - gcsNameTransformer.convertStreamName(streamId.rawName) - ) - ) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperation.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperation.kt deleted file mode 100644 index ce3c6037cb1c5..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/operation/BigQueryStorageOperation.kt +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.operation - -import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.QueryJobConfiguration -import com.google.cloud.bigquery.TableId -import com.google.cloud.bigquery.TableResult -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.integrations.base.destination.operation.StorageOperation -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig -import io.airbyte.integrations.base.destination.typing_deduping.StreamId -import io.airbyte.integrations.base.destination.typing_deduping.TyperDeduperUtil -import io.airbyte.integrations.destination.bigquery.BigQueryUtils -import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.github.oshai.kotlinlogging.KotlinLogging -import java.time.Instant -import java.util.Optional -import java.util.concurrent.ConcurrentHashMap - -private val log = KotlinLogging.logger {} - -abstract class BigQueryStorageOperation( - protected val bigquery: BigQuery, - private val sqlGenerator: BigQuerySqlGenerator, - private val destinationHandler: BigQueryDestinationHandler, - protected val datasetLocation: String -) : StorageOperation { - private val existingSchemas = ConcurrentHashMap.newKeySet() - override fun prepareStage(streamId: StreamId, suffix: String, replace: Boolean) { - // Prepare staging table. For overwrite, it does drop-create so we can skip explicit create. - if (replace) { - truncateStagingTable(streamId, suffix) - } else { - createStagingTable(streamId, suffix) - } - } - - override fun overwriteStage(streamId: StreamId, suffix: String) { - if (suffix == "") { - throw IllegalArgumentException("Cannot overwrite raw table with empty suffix") - } - bigquery.delete(tableId(streamId, "")) - bigquery.query( - QueryJobConfiguration.of( - """ALTER TABLE `${streamId.rawNamespace}`.`${streamId.rawName}$suffix` RENAME TO `${streamId.rawName}`""" - ), - ) - } - - override fun transferFromTempStage(streamId: StreamId, suffix: String) { - if (suffix == "") { - throw IllegalArgumentException( - "Cannot transfer records from temp raw table with empty suffix" - ) - } - // TODO figure out how to make this work - // something about incompatible partitioning spec (probably b/c we're copying from a temp - // table partitioned on generation ID into an old real raw table partitioned on - // extracted_at) - val tempRawTable = tableId(streamId, suffix) - // val jobConf = - // CopyJobConfiguration.newBuilder(tableId(streamId, ""), tempRawTable) - // .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND) - // .build() - // val job = bigquery.create(JobInfo.of(jobConf)) - // BigQueryUtils.waitForJobFinish(job) - - bigquery.query( - QueryJobConfiguration.of( - """ - INSERT INTO `${streamId.rawNamespace}`.`${streamId.rawName}` - SELECT * FROM `${streamId.rawNamespace}`.`${streamId.rawName}$suffix` - """.trimIndent() - ) - ) - bigquery.delete(tempRawTable) - } - - override fun getStageGeneration(streamId: StreamId, suffix: String): Long? { - val result: TableResult = - bigquery.query( - QueryJobConfiguration.of( - "SELECT _airbyte_generation_id FROM ${streamId.rawNamespace}.${streamId.rawName}$suffix LIMIT 1" - ), - ) - if (result.totalRows == 0L) { - return null - } - val value = result.iterateAll().first().get(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID) - return if (value == null || value.isNull) { - 0 - } else { - value.longValue - } - } - - private fun createStagingTable(streamId: StreamId, suffix: String) { - BigQueryUtils.createPartitionedTableIfNotExists( - bigquery, - tableId(streamId, suffix), - BigQueryRecordFormatter.SCHEMA_V2, - ) - } - - private fun dropStagingTable(streamId: StreamId, suffix: String) { - bigquery.delete(tableId(streamId, suffix)) - } - - /** - * "Truncates" table, this is a workaround to the issue with TRUNCATE TABLE in BigQuery where - * the table's partition filter must be turned off to truncate. Since deleting a table is a free - * operation this option re-uses functions that already exist - */ - private fun truncateStagingTable(streamId: StreamId, suffix: String) { - val tableId = TableId.of(streamId.rawNamespace, streamId.rawName) - log.info { "Truncating raw table $tableId" } - dropStagingTable(streamId, suffix) - createStagingTable(streamId, suffix) - } - - override fun cleanupStage(streamId: StreamId) { - log.info { "Nothing to cleanup in stage for Streaming inserts" } - } - - override fun createFinalTable(streamConfig: StreamConfig, suffix: String, replace: Boolean) { - destinationHandler.execute(sqlGenerator.createTable(streamConfig, suffix, replace)) - } - - override fun softResetFinalTable(streamConfig: StreamConfig) { - TyperDeduperUtil.executeSoftReset( - sqlGenerator = sqlGenerator, - destinationHandler = destinationHandler, - streamConfig, - ) - } - - override fun overwriteFinalTable(streamConfig: StreamConfig, tmpTableSuffix: String) { - if (tmpTableSuffix == "") { - throw IllegalArgumentException("Cannot overwrite final table with empty suffix") - } - log.info { - "Overwriting table ${streamConfig.id.finalTableId(BigQuerySqlGenerator.QUOTE)} with ${ - streamConfig.id.finalTableId( - BigQuerySqlGenerator.QUOTE, - tmpTableSuffix, - ) - }" - } - destinationHandler.execute( - sqlGenerator.overwriteFinalTable(streamConfig.id, tmpTableSuffix), - ) - } - - override fun typeAndDedupe( - streamConfig: StreamConfig, - maxProcessedTimestamp: Optional, - finalTableSuffix: String - ) { - TyperDeduperUtil.executeTypeAndDedupe( - sqlGenerator = sqlGenerator, - destinationHandler = destinationHandler, - streamConfig, - maxProcessedTimestamp, - finalTableSuffix, - ) - } - - companion object { - fun tableId(streamId: StreamId, suffix: String = ""): TableId = - TableId.of(streamId.rawNamespace, streamId.rawName + suffix) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt index 178931dc4f7d6..cdf80efc5a42d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt @@ -23,6 +23,8 @@ import io.github.oshai.kotlinlogging.KotlinLogging import java.util.UUID import java.util.stream.Stream import kotlin.math.min +import kotlinx.coroutines.coroutineScope +import kotlinx.coroutines.launch private val logger = KotlinLogging.logger {} @@ -110,16 +112,22 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo } } - override fun createNamespaces(namespaces: List) { - namespaces.forEach { dataset -> - logger.info { "Creating dataset if not present $dataset" } - try { - BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation) - } catch (e: BigQueryException) { - if (ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code)) { - throw ConfigErrorException(e.message!!, e) - } else { - throw e + override suspend fun createNamespaces(namespaces: List) { + coroutineScope { + namespaces.forEach { dataset -> + launch { + logger.info { "Creating dataset if not present $dataset" } + try { + BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation) + } catch (e: BigQueryException) { + if ( + ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code) + ) { + throw ConfigErrorException(e.message!!, e) + } else { + throw e + } + } } } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 3d12658f165f7..bd90365924916 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -49,8 +49,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo airbyteType: AirbyteType, forceSafeCast: Boolean ): String { - // TODO this should actually be LegacyUnionType - if (airbyteType is UnionType) { + if (airbyteType is UnionType && airbyteType.isLegacyUnion) { // This is guaranteed to not be a Union, so we won't recurse infinitely val chosenType: AirbyteType = airbyteType.chooseType() return extractAndCast(columnName, chosenType, forceSafeCast) @@ -629,9 +628,8 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo const val QUOTE: String = "`" val nameTransformer = BigQuerySQLNameTransformer() - @JvmStatic - fun toDialectType(type: AirbyteType): StandardSQLTypeName { - return when (type) { + fun toDialectType(type: AirbyteType): StandardSQLTypeName = + when (type) { BooleanType -> StandardSQLTypeName.BOOL DateType -> StandardSQLTypeName.DATE IntegerType -> StandardSQLTypeName.INT64 @@ -646,11 +644,14 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo is ObjectType, ObjectTypeWithEmptySchema, ObjectTypeWithoutSchema, - // TODO handle LegacyUnion type (do the Union.chooseType thing) - is UnionType, + is UnionType -> + if ((type as UnionType).isLegacyUnion) { + toDialectType(type.chooseType()) + } else { + StandardSQLTypeName.JSON + } is UnknownType -> StandardSQLTypeName.JSON } - } fun clusteringColumns( stream: DestinationStream, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index e528e1a3fbf45..5010fa23f424f 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -53,13 +53,10 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : private fun getInitialRawTableState( rawTableName: TableName, suffix: String - ): RawTableInitialStatus { + ): RawTableInitialStatus? { bq.getTable(TableId.of(rawTableName.namespace, rawTableName.name + suffix)) - ?: // Table doesn't exist. There are no unprocessed records, and no timestamp. - return RawTableInitialStatus( - hasUnprocessedRecords = false, - maxProcessedTimestamp = null, - ) + // Table doesn't exist. There are no unprocessed records, and no timestamp. + ?: return null val rawTableIdQuoted = """`${rawTableName.namespace}`.`${rawTableName.name}$suffix`""" val unloadedRecordTimestamp = @@ -79,8 +76,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : .first() // If this value is null, then there are no records with null loaded_at. // If it's not null, then we can return immediately - we've found some unprocessed records - // and their - // timestamp. + // and their timestamp. if (!unloadedRecordTimestamp.isNull) { return RawTableInitialStatus( hasUnprocessedRecords = true, @@ -183,7 +179,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : val streamSchema: Map = (stream.schema as ObjectType).properties.entries.associate { - it.key to BigQuerySqlGenerator.toDialectType(it.value) + it.key to BigQuerySqlGenerator.toDialectType(it.value.type) } val existingSchema = diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt deleted file mode 100644 index ae7859c42fec8..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryInitialStateGatherer.kt +++ /dev/null @@ -1,20 +0,0 @@ -/* - * Copyright (c) 2025 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.write - -import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus - -class BigqueryInitialStateGatherer : - DestinationInitialStatusGatherer { - override fun gatherInitialStatus( - streams: Map> - ): Map { - TODO("Not yet implemented") - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt index ba9cfb5a5102b..30fd0e151ab6d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt @@ -1,26 +1,115 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.integrations.destination.bigquery.write +import com.google.cloud.bigquery.BigQuery +import com.google.cloud.bigquery.QueryJobConfiguration +import com.google.cloud.bigquery.TableId +import com.google.cloud.bigquery.TableResult +import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingRawTableOperations +import io.airbyte.integrations.destination.bigquery.BigQueryUtils +import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter +import io.github.oshai.kotlinlogging.KotlinLogging + +private val logger = KotlinLogging.logger {} -class BigqueryRawTableOperations : TypingDedupingRawTableOperations { +class BigqueryRawTableOperations(private val bigquery: BigQuery) : + TypingDedupingRawTableOperations { override fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean) { - TODO("Not yet implemented") + // Prepare staging table. For overwrite, it does drop-create so we can skip explicit create. + if (replace) { + truncateStagingTable(rawTableName, suffix) + } else { + createStagingTable(rawTableName, suffix) + } } override fun overwriteRawTable(rawTableName: TableName, suffix: String) { - TODO("Not yet implemented") + if (suffix == "") { + throw IllegalArgumentException("Cannot overwrite raw table with empty suffix") + } + bigquery.delete(tableId(rawTableName, "")) + bigquery.query( + QueryJobConfiguration.of( + """ALTER TABLE `${rawTableName.namespace}`.`${rawTableName.name}$suffix` RENAME TO `${rawTableName.name}`""" + ), + ) } override fun transferFromTempRawTable(rawTableName: TableName, suffix: String) { - TODO("Not yet implemented") + if (suffix == "") { + throw IllegalArgumentException( + "Cannot transfer records from temp raw table with empty suffix" + ) + } + // TODO figure out how to make this work + // something about incompatible partitioning spec (probably b/c we're copying from a temp + // table partitioned on generation ID into an old real raw table partitioned on + // extracted_at) + val tempRawTable = tableId(rawTableName, suffix) + // val jobConf = + // CopyJobConfiguration.newBuilder(tableId(streamId, ""), tempRawTable) + // .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND) + // .build() + // val job = bigquery.create(JobInfo.of(jobConf)) + // BigQueryUtils.waitForJobFinish(job) + + bigquery.query( + QueryJobConfiguration.of( + """ + INSERT INTO `${rawTableName.namespace}`.`${rawTableName.name}` + SELECT * FROM `${rawTableName.namespace}`.`${rawTableName.name}$suffix` + """.trimIndent() + ) + ) + bigquery.delete(tempRawTable) } override fun getRawTableGeneration(rawTableName: TableName, suffix: String): Long? { - TODO("Not yet implemented") + val result: TableResult = + bigquery.query( + QueryJobConfiguration.of( + "SELECT _airbyte_generation_id FROM ${rawTableName.namespace}.${rawTableName.name}$suffix LIMIT 1" + ), + ) + if (result.totalRows == 0L) { + return null + } + val value = result.iterateAll().first().get(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID) + return if (value == null || value.isNull) { + 0 + } else { + value.longValue + } + } + + private fun createStagingTable(rawTableName: TableName, suffix: String) { + BigQueryUtils.createPartitionedTableIfNotExists( + bigquery, + tableId(rawTableName, suffix), + BigQueryRecordFormatter.SCHEMA_V2, + ) + } + + private fun dropStagingTable(rawTableName: TableName, suffix: String) { + bigquery.delete(tableId(rawTableName, suffix)) } - override fun cleanupStage(rawTableName: TableName) { - TODO("Not yet implemented") + /** + * "Truncates" table, this is a workaround to the issue with TRUNCATE TABLE in BigQuery where + * the table's partition filter must be turned off to truncate. Since deleting a table is a free + * operation this option re-uses functions that already exist + */ + private fun truncateStagingTable(rawTableName: TableName, suffix: String) { + logger.info { "Truncating raw table ${tableId(rawTableName, suffix)}" } + dropStagingTable(rawTableName, suffix) + createStagingTable(rawTableName, suffix) } + + private fun tableId(rawTableName: TableName, suffix: String = ""): TableId = + TableId.of(rawTableName.namespace, rawTableName.name + suffix) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index bb971abe88589..c9541ec32ef9e 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -14,6 +14,10 @@ import io.airbyte.integrations.base.destination.typing_deduping.StreamId import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryColumnNameGenerator +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDestinationInitialStatusGatherer +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator import io.micronaut.context.annotation.Factory import jakarta.inject.Singleton @@ -29,9 +33,14 @@ class BigqueryWriterFactory( fun make() = TypingDedupingWriter( catalog, - BigqueryInitialStateGatherer(), - BigqueryRawTableOperations(), + BigqueryRawTableNameGenerator(config.datasetId, config.rawTableDataset), + BigqueryFinalTableNameGenerator(config.datasetId), + BigqueryColumnNameGenerator(), + BigqueryDestinationInitialStatusGatherer(bigquery), + destinationHandler, + BigqueryRawTableOperations(bigquery), TypingDedupingFinalTableOperations(sqlGenerator, destinationHandler), + disableTypeDedupe = config.disableTypingDeduping, ) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index f0baf5d9d38b3..e20839c0e0990 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -18,9 +18,6 @@ import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation.Companion.CONFIG_ERROR_MSG -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation.Companion.HTTP_STATUS_CODE_FORBIDDEN -import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation.Companion.HTTP_STATUS_CODE_NOT_FOUND import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.write.TempUtils @@ -102,4 +99,18 @@ class BigqueryBatchStandardInsertsLoaderFactory( return BigqueryBatchStandardInsertsLoader(writer) } + + companion object { + const val HTTP_STATUS_CODE_FORBIDDEN = 403 + const val HTTP_STATUS_CODE_NOT_FOUND = 404 + + val CONFIG_ERROR_MSG = + """ + |Failed to write to destination schema. + | 1. Make sure you have all required permissions for writing to the schema. + | 2. Make sure that the actual destination schema's location corresponds to location provided in connector's config. + | 3. Try to change the "Destination schema" from "Mirror Source Structure" (if it's set) tp the "Destination Default" option. + |More details: + |""".trimMargin() + } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt index 34e421dd955dc..9a07f16dfa8bb 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt @@ -62,7 +62,21 @@ class StandardInsertRawOverrideDisableTd : } } -// @Disabled("Disabling until we have the full flow") +class StandardInsertRawOverride : + BigqueryWriteTest( + BigQueryDestinationTestUtils.createConfig( + configFile = Path.of("secrets/credentials-1s1t-standard-raw-override.json"), + datasetId = DEFAULT_NAMESPACE_PLACEHOLDER, + stagingPath = "test_path/$DEFAULT_NAMESPACE_PLACEHOLDER", + ) + .serializeToString(), + ) { + @Test + override fun testBasicWrite() { + super.testBasicWrite() + } +} + class GcsRawOverrideDisableTd : BigqueryWriteTest( BigQueryDestinationTestUtils.createConfig( From c89574e530bbefbc70bfafab76ca516b70cf7bd3 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 09:46:57 -0700 Subject: [PATCH 19/55] resources --- .../main/resources/application-connector.yaml | 5 + .../src/main/resources/spec.json | 239 ------------------ 2 files changed, 5 insertions(+), 239 deletions(-) create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/resources/application-connector.yaml delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/resources/spec.json diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/resources/application-connector.yaml b/airbyte-integrations/connectors/destination-bigquery/src/main/resources/application-connector.yaml new file mode 100644 index 0000000000000..b5cd0699fa97a --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/resources/application-connector.yaml @@ -0,0 +1,5 @@ +airbyte: + destination: + core: + types: + unions: LEGACY diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/resources/spec.json b/airbyte-integrations/connectors/destination-bigquery/src/main/resources/spec.json deleted file mode 100644 index 07d045485c358..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/resources/spec.json +++ /dev/null @@ -1,239 +0,0 @@ -{ - "documentationUrl": "https://docs.airbyte.com/integrations/destinations/bigquery", - "supportsIncremental": true, - "supportsDBT": true, - "supported_destination_sync_modes": ["overwrite", "append", "append_dedup"], - "connectionSpecification": { - "$schema": "http://json-schema.org/draft-07/schema#", - "title": "BigQuery Destination Spec", - "type": "object", - "required": ["project_id", "dataset_location", "dataset_id"], - "additionalProperties": true, - "properties": { - "project_id": { - "type": "string", - "description": "The GCP project ID for the project containing the target BigQuery dataset. Read more here.", - "title": "Project ID", - "group": "connection", - "order": 0 - }, - "dataset_location": { - "type": "string", - "description": "The location of the dataset. Warning: Changes made after creation will not be applied. Read more here.", - "title": "Dataset Location", - "group": "connection", - "order": 1, - "enum": [ - "US", - "EU", - "asia-east1", - "asia-east2", - "asia-northeast1", - "asia-northeast2", - "asia-northeast3", - "asia-south1", - "asia-south2", - "asia-southeast1", - "asia-southeast2", - "australia-southeast1", - "australia-southeast2", - "europe-central1", - "europe-central2", - "europe-north1", - "europe-southwest1", - "europe-west1", - "europe-west2", - "europe-west3", - "europe-west4", - "europe-west6", - "europe-west7", - "europe-west8", - "europe-west9", - "europe-west12", - "me-central1", - "me-central2", - "me-west1", - "northamerica-northeast1", - "northamerica-northeast2", - "southamerica-east1", - "southamerica-west1", - "us-central1", - "us-east1", - "us-east2", - "us-east3", - "us-east4", - "us-east5", - "us-south1", - "us-west1", - "us-west2", - "us-west3", - "us-west4" - ] - }, - "dataset_id": { - "type": "string", - "description": "The default BigQuery Dataset ID that tables are replicated to if the source does not specify a namespace. Read more here.", - "title": "Default Dataset ID", - "group": "connection", - "order": 2 - }, - "loading_method": { - "type": "object", - "title": "Loading Method", - "description": "The way data will be uploaded to BigQuery.", - "display_type": "radio", - "group": "connection", - "order": 3, - "oneOf": [ - { - "title": "Batched Standard Inserts", - "required": ["method"], - "description": "Direct loading using batched SQL INSERT statements. This method uses the BigQuery driver to convert large INSERT statements into file uploads automatically.", - "properties": { - "method": { - "type": "string", - "const": "Standard" - } - } - }, - { - "title": "GCS Staging", - "description": "Writes large batches of records to a file, uploads the file to GCS, then uses COPY INTO to load your data into BigQuery.", - "required": [ - "method", - "gcs_bucket_name", - "gcs_bucket_path", - "credential" - ], - "properties": { - "method": { - "type": "string", - "const": "GCS Staging" - }, - "credential": { - "title": "Credential", - "description": "An HMAC key is a type of credential and can be associated with a service account or a user account in Cloud Storage. Read more here.", - "type": "object", - "order": 1, - "oneOf": [ - { - "title": "HMAC key", - "required": [ - "credential_type", - "hmac_key_access_id", - "hmac_key_secret" - ], - "properties": { - "credential_type": { - "type": "string", - "const": "HMAC_KEY", - "order": 0 - }, - "hmac_key_access_id": { - "type": "string", - "description": "HMAC key access ID. When linked to a service account, this ID is 61 characters long; when linked to a user account, it is 24 characters long.", - "title": "HMAC Key Access ID", - "airbyte_secret": true, - "examples": ["1234567890abcdefghij1234"], - "order": 1 - }, - "hmac_key_secret": { - "type": "string", - "description": "The corresponding secret for the access ID. It is a 40-character base-64 encoded string.", - "title": "HMAC Key Secret", - "airbyte_secret": true, - "examples": [ - "1234567890abcdefghij1234567890ABCDEFGHIJ" - ], - "order": 2 - } - } - } - ] - }, - "gcs_bucket_name": { - "title": "GCS Bucket Name", - "type": "string", - "description": "The name of the GCS bucket. Read more here.", - "examples": ["airbyte_sync"], - "order": 2 - }, - "gcs_bucket_path": { - "title": "GCS Bucket Path", - "description": "Directory under the GCS bucket where data will be written.", - "type": "string", - "examples": ["data_sync/test"], - "order": 3 - }, - "keep_files_in_gcs-bucket": { - "type": "string", - "description": "This upload method is supposed to temporary store records in GCS bucket. By this select you can chose if these records should be removed from GCS when migration has finished. The default \"Delete all tmp files from GCS\" value is used if not set explicitly.", - "title": "GCS Tmp Files Afterward Processing", - "default": "Delete all tmp files from GCS", - "enum": [ - "Delete all tmp files from GCS", - "Keep all tmp files in GCS" - ], - "order": 4 - } - } - } - ] - }, - "credentials_json": { - "type": "string", - "description": "The contents of the JSON service account key. Check out the docs if you need help generating this key. Default credentials will be used if this field is left empty.", - "title": "Service Account Key JSON (Required for cloud, optional for open-source)", - "airbyte_secret": true, - "group": "connection", - "order": 4, - "always_show": true - }, - "transformation_priority": { - "type": "string", - "description": "Interactive run type means that the query is executed as soon as possible, and these queries count towards concurrent rate limit and daily limit. Read more about interactive run type here. Batch queries are queued and started as soon as idle resources are available in the BigQuery shared resource pool, which usually occurs within a few minutes. Batch queries don’t count towards your concurrent rate limit. Read more about batch queries here. The default \"interactive\" value is used if not set explicitly.", - "title": "Transformation Query Run Type", - "default": "interactive", - "enum": ["interactive", "batch"], - "order": 5, - "group": "advanced" - }, - "big_query_client_buffer_size_mb": { - "title": "Google BigQuery Client Chunk Size", - "description": "Google BigQuery client's chunk (buffer) size (MIN=1, MAX = 15) for each table. The size that will be written by a single RPC. Written data will be buffered and only flushed upon reaching this size or closing the channel. The default 15MB value is used if not set explicitly. Read more here.", - "type": "integer", - "minimum": 1, - "maximum": 15, - "default": 15, - "examples": ["15"], - "order": 6, - "group": "advanced" - }, - "raw_data_dataset": { - "type": "string", - "description": "The dataset to write raw tables into (default: airbyte_internal)", - "title": "Raw Table Dataset Name", - "order": 7, - "group": "advanced" - }, - "disable_type_dedupe": { - "type": "boolean", - "default": false, - "description": "Disable Writing Final Tables. WARNING! The data format in _airbyte_data is likely stable but there are no guarantees that other metadata columns will remain the same in future versions", - "title": "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)", - "order": 8, - "group": "advanced" - } - }, - "groups": [ - { - "id": "connection", - "title": "Connection" - }, - { - "id": "advanced", - "title": "Advanced" - } - ] - } -} From 09d24970d09d39108f47aa5301d926b0d67e4c27 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 09:56:18 -0700 Subject: [PATCH 20/55] less micronaut --- .../util/BigqueryDestinationHandlerFactory.kt | 17 ----------------- .../util/BigquerySqlGeneratorFactory.kt | 13 ------------- .../bigquery/write/BigqueryWriter.kt | 13 ++++++++----- .../BigqueryBatchStandardInsertLoader.kt | 2 +- 4 files changed, 9 insertions(+), 36 deletions(-) delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt deleted file mode 100644 index 8f2b328f56e97..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigqueryDestinationHandlerFactory.kt +++ /dev/null @@ -1,17 +0,0 @@ -package io.airbyte.integrations.destination.bigquery.util - -import com.google.cloud.bigquery.BigQuery -import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler -import io.micronaut.context.annotation.Factory -import jakarta.inject.Singleton - -@Factory -class BigqueryDestinationHandlerFactory( - private val bigquery: BigQuery, - private val config: BigqueryConfiguration, -) { - @Singleton - fun make(): BigQueryDestinationHandler = - BigQueryDestinationHandler(bigquery, config.datasetLocation.region) -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt deleted file mode 100644 index 0d098c842ee8e..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/util/BigquerySqlGeneratorFactory.kt +++ /dev/null @@ -1,13 +0,0 @@ -package io.airbyte.integrations.destination.bigquery.util - -import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.micronaut.context.annotation.Factory -import jakarta.inject.Singleton - -@Factory -class BigquerySqlGeneratorFactory(private val config: BigqueryConfiguration) { - @Singleton - fun make(): BigQuerySqlGenerator = - BigQuerySqlGenerator(config.projectId, config.datasetLocation.region) -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index c9541ec32ef9e..7e0392702aefa 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -26,12 +26,11 @@ class BigqueryWriterFactory( private val catalog: DestinationCatalog, private val bigquery: BigQuery, private val config: BigqueryConfiguration, - private val sqlGenerator: BigQuerySqlGenerator, - private val destinationHandler: BigQueryDestinationHandler, ) { @Singleton - fun make() = - TypingDedupingWriter( + fun make(): TypingDedupingWriter { + val destinationHandler = BigQueryDestinationHandler(bigquery, config.datasetLocation.region) + return TypingDedupingWriter( catalog, BigqueryRawTableNameGenerator(config.datasetId, config.rawTableDataset), BigqueryFinalTableNameGenerator(config.datasetId), @@ -39,9 +38,13 @@ class BigqueryWriterFactory( BigqueryDestinationInitialStatusGatherer(bigquery), destinationHandler, BigqueryRawTableOperations(bigquery), - TypingDedupingFinalTableOperations(sqlGenerator, destinationHandler), + TypingDedupingFinalTableOperations( + BigQuerySqlGenerator(config.projectId, config.datasetLocation.region), + destinationHandler, + ), disableTypeDedupe = config.disableTypingDeduping, ) + } } // TODO delete this - this is definitely duplicated code, and also is definitely wrong diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index e20839c0e0990..0226263a1f19f 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -69,7 +69,7 @@ class BigqueryBatchStandardInsertsLoaderFactory( ) : DirectLoaderFactory { override fun create( streamDescriptor: DestinationStream.Descriptor, - part: Int + part: Int, ): BigqueryBatchStandardInsertsLoader { val writeChannelConfiguration = // TODO need to write to raw vs final table appropriately From a852bef0b96bd158a0cbf58d6d632efae2b68aa0 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 11:34:26 -0700 Subject: [PATCH 21/55] kill TempUtils --- .../load/orchestration/DestinationNames.kt | 6 ++- .../TableNameMapFactory.kt | 52 +++++++++++++++++++ .../TypingDedupingWriter.kt | 27 +--------- .../typing_deduping/BigqueryNameGenerators.kt | 28 +++++----- .../bigquery/write/BigqueryWriter.kt | 34 ++---------- .../write/bulk_loader/BigQueryBulkLoader.kt | 19 ++++--- .../BigqueryBatchStandardInsertLoader.kt | 9 ++-- 7 files changed, 95 insertions(+), 80 deletions(-) create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt index db3d635f2b690..3589f2383026a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt @@ -55,10 +55,14 @@ fun TableName?.conflictsWith(other: TableName?): Boolean { value class ColumnNameMapping(private val columnNameMapping: Map) : Map by columnNameMapping -fun interface TableNameGenerator { +sealed interface TableNameGenerator { fun getTableName(streamDescriptor: DestinationStream.Descriptor): TableName } +fun interface RawTableNameGenerator : TableNameGenerator + +fun interface FinalTableNameGenerator : TableNameGenerator + fun interface ColumnNameGenerator { /** * In some database/warehouses, there's a difference between how a name is _displayed_, and how diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt new file mode 100644 index 0000000000000..65bf1ab9d0d92 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt @@ -0,0 +1,52 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import io.airbyte.cdk.load.command.DestinationCatalog +import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameGenerator +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.FinalTableNameGenerator +import io.airbyte.cdk.load.orchestration.RawTableNameGenerator +import io.airbyte.cdk.load.orchestration.TableNames +import io.micronaut.context.annotation.Factory +import javax.inject.Singleton + +@Factory +class TypingDedupingTableNameMapFactory( + private val catalog: DestinationCatalog, + private val rawTableNameGenerator: RawTableNameGenerator, + private val finalTableNameGenerator: FinalTableNameGenerator, + private val finalTableColumnNameGenerator: ColumnNameGenerator, +) { + @Singleton + fun get(): Map> { + // TODO handle collisions in table names + return catalog.streams.associateWith { stream -> + Pair( + TableNames( + rawTableName = rawTableNameGenerator.getTableName(stream.descriptor), + finalTableName = finalTableNameGenerator.getTableName(stream.descriptor), + ), + ColumnNameMapping( + // TODO handle collisions in column names + stream.schema.asColumns().mapValues { (columnName, _) -> + finalTableColumnNameGenerator.getColumnName(columnName).displayName + } + ) + ) + } + } +} + +@Factory +class TypingDedupingTableNameMapByDescriptorFactory( + private val map: Map>, +) { + @Singleton + fun get(): Map> { + return map.mapKeys { (k, _) -> k.descriptor } + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index 46307cd8a29de..a1e25761dc18b 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -4,13 +4,10 @@ package io.airbyte.cdk.load.orchestration.legacy_typing_deduping -import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameGenerator import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationHandler import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer -import io.airbyte.cdk.load.orchestration.TableNameGenerator import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader @@ -20,10 +17,7 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.runBlocking class TypingDedupingWriter( - private val catalog: DestinationCatalog, - private val rawTableTableNameGenerator: TableNameGenerator, - private val finalTableTableNameGenerator: TableNameGenerator, - private val finalTableColumnNameGenerator: ColumnNameGenerator, + private val names: Map>, private val stateGatherer: DestinationInitialStatusGatherer, private val destinationHandler: DestinationHandler, @@ -31,29 +25,10 @@ class TypingDedupingWriter( private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, ) : DestinationWriter { - private lateinit var names: Map> private lateinit var initialStatuses: Map override suspend fun setup() { - // TODO handle collisions in table names - names = - catalog.streams.associateWith { stream -> - Pair( - TableNames( - rawTableName = rawTableTableNameGenerator.getTableName(stream.descriptor), - finalTableName = - finalTableTableNameGenerator.getTableName(stream.descriptor), - ), - ColumnNameMapping( - // TODO handle collisions in column names - stream.schema.asColumns().mapValues { (columnName, _) -> - finalTableColumnNameGenerator.getColumnName(columnName).displayName - } - ) - ) - } - Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> destinationHandler.createNamespaces( names.values.map { (tableNames, _) -> tableNames.rawTableName!!.namespace } + diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt index 3a6e9d9153833..6a8d6c8f2b2ed 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt @@ -6,37 +6,39 @@ package io.airbyte.integrations.destination.bigquery.typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.ColumnNameGenerator +import io.airbyte.cdk.load.orchestration.FinalTableNameGenerator +import io.airbyte.cdk.load.orchestration.RawTableNameGenerator import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNameGenerator import io.airbyte.integrations.base.destination.typing_deduping.StreamId +import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer import java.util.Locale +import javax.inject.Singleton -class BigqueryRawTableNameGenerator( - private val defaultNamespace: String, - private val rawNamespace: String, -) : TableNameGenerator { - override fun getTableName(streamDescriptor: DestinationStream.Descriptor): TableName { - return TableName( - nameTransformer.getNamespace(rawNamespace), +@Singleton +class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTableNameGenerator { + override fun getTableName(streamDescriptor: DestinationStream.Descriptor) = + TableName( + nameTransformer.getNamespace(config.rawTableDataset), nameTransformer.convertStreamName( StreamId.concatenateRawTableName( - streamDescriptor.namespace ?: defaultNamespace, - streamDescriptor.name + streamDescriptor.namespace ?: config.datasetId, + streamDescriptor.name, ) ), ) - } } -class BigqueryFinalTableNameGenerator(private val defaultNamespace: String) : TableNameGenerator { +@Singleton +class BigqueryFinalTableNameGenerator(val config: BigqueryConfiguration) : FinalTableNameGenerator { override fun getTableName(streamDescriptor: DestinationStream.Descriptor) = TableName( - nameTransformer.getNamespace(streamDescriptor.namespace ?: defaultNamespace), + nameTransformer.getNamespace(streamDescriptor.namespace ?: config.datasetId), nameTransformer.convertStreamName(streamDescriptor.name), ) } +@Singleton class BigqueryColumnNameGenerator : ColumnNameGenerator { override fun getColumnName(column: String): ColumnNameGenerator.ColumnName { return ColumnNameGenerator.ColumnName( diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index 7e0392702aefa..3e9df98be8b4e 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -5,36 +5,29 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.TableId -import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingFinalTableOperations import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingWriter -import io.airbyte.integrations.base.destination.typing_deduping.StreamId import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryColumnNameGenerator import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDestinationInitialStatusGatherer -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator import io.micronaut.context.annotation.Factory import jakarta.inject.Singleton @Factory class BigqueryWriterFactory( - private val catalog: DestinationCatalog, private val bigquery: BigQuery, private val config: BigqueryConfiguration, + private val names: Map>, ) { @Singleton fun make(): TypingDedupingWriter { val destinationHandler = BigQueryDestinationHandler(bigquery, config.datasetLocation.region) return TypingDedupingWriter( - catalog, - BigqueryRawTableNameGenerator(config.datasetId, config.rawTableDataset), - BigqueryFinalTableNameGenerator(config.datasetId), - BigqueryColumnNameGenerator(), + names, BigqueryDestinationInitialStatusGatherer(bigquery), destinationHandler, BigqueryRawTableOperations(bigquery), @@ -46,22 +39,3 @@ class BigqueryWriterFactory( ) } } - -// TODO delete this - this is definitely duplicated code, and also is definitely wrong -// e.g. we need to handle special chars in stream name/namespace (c.f. -// bigquerysqlgenerator.buildStreamId) -// and that logic needs to be in BigqueryWriter.setup, to handle collisions -// (probably actually a toolkit) -object TempUtils { - fun rawTableId( - config: BigqueryConfiguration, - streamDescriptor: DestinationStream.Descriptor, - ) = - TableId.of( - config.rawTableDataset, - StreamId.concatenateRawTableName( - streamDescriptor.namespace ?: config.datasetId, - streamDescriptor.name - ) - ) -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt index 374a613cf0a1a..fcdf4a187332d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt @@ -8,11 +8,13 @@ import com.google.cloud.bigquery.* import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.LoadJobConfiguration -import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.file.gcs.GcsBlob import io.airbyte.cdk.load.file.gcs.GcsClient import io.airbyte.cdk.load.message.StreamKey +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.write.db.BulkLoader import io.airbyte.cdk.load.write.db.BulkLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -20,7 +22,6 @@ import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordForm import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration -import io.airbyte.integrations.destination.bigquery.write.TempUtils import io.micronaut.context.annotation.Requires import io.micronaut.context.condition.Condition import io.micronaut.context.condition.ConditionContext @@ -30,10 +31,10 @@ class BigQueryBulkLoader( private val storageClient: GcsClient, private val bigQueryClient: BigQuery, private val bigQueryConfiguration: BigqueryConfiguration, - private val stream: DestinationStream, + private val rawTableName: TableName, ) : BulkLoader { override suspend fun load(remoteObject: GcsBlob) { - val rawTableId = TempUtils.rawTableId(bigQueryConfiguration, stream.descriptor) + val rawTableId = TableId.of(rawTableName.namespace, rawTableName.name) val gcsUri = "gs://${remoteObject.storageConfig.gcsBucketName}/${remoteObject.key}" val csvOptions = @@ -84,7 +85,7 @@ class BigqueryConfiguredForBulkLoad : Condition { @Singleton @Requires(condition = BigqueryConfiguredForBulkLoad::class) class BigQueryBulkLoaderFactory( - private val catalog: DestinationCatalog, + private val names: Map>, private val storageClient: GcsClient, private val bigQueryClient: BigQuery, private val bigQueryConfiguration: BigqueryConfiguration @@ -98,7 +99,11 @@ class BigQueryBulkLoaderFactory( override val maxMemoryRatioReservedForParts: Double = 0.6 override fun create(key: StreamKey, partition: Int): BulkLoader { - val stream = catalog.getStream(key.stream) - return BigQueryBulkLoader(storageClient, bigQueryClient, bigQueryConfiguration, stream) + return BigQueryBulkLoader( + storageClient, + bigQueryClient, + bigQueryConfiguration, + names[key.stream]!!.first.rawTableName!!, + ) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index 0226263a1f19f..6030a658360dd 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -10,17 +10,19 @@ import com.google.cloud.bigquery.FormatOptions import com.google.cloud.bigquery.JobId import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.TableDataWriteChannel +import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.WriteChannelConfiguration import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.message.DestinationRecordRaw +import io.airbyte.cdk.load.orchestration.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.write.TempUtils import io.micronaut.context.annotation.Requires import io.micronaut.context.condition.Condition import io.micronaut.context.condition.ConditionContext @@ -66,14 +68,15 @@ class BigqueryConfiguredForBatchStandardInserts : Condition { class BigqueryBatchStandardInsertsLoaderFactory( private val bigquery: BigQuery, private val config: BigqueryConfiguration, + private val names: Map>, ) : DirectLoaderFactory { override fun create( streamDescriptor: DestinationStream.Descriptor, part: Int, ): BigqueryBatchStandardInsertsLoader { + val tableName = names[streamDescriptor]!!.first.rawTableName!! val writeChannelConfiguration = - // TODO need to write to raw vs final table appropriately - WriteChannelConfiguration.newBuilder(TempUtils.rawTableId(config, streamDescriptor)) + WriteChannelConfiguration.newBuilder(TableId.of(tableName.namespace, tableName.name)) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) .setSchema(BigQueryRecordFormatter.SCHEMA_V2) .setFormatOptions(FormatOptions.json()) From 9e4b482ad31f48795d4b9958d80bb584e9b404af Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 12:00:53 -0700 Subject: [PATCH 22/55] delete some code --- ...stractBigQueryDestinationAcceptanceTest.kt | 2 +- .../bigquery/BigQueryDestinationTest.kt | 2 +- .../AbstractBigQueryTypingDedupingTest.kt | 3 +- .../bigquery/BigQueryDestination.kt | 525 ------------------ .../bigquery/BigQueryDestinationTestUtils.kt | 20 +- .../destination/bigquery/BigqueryWriteTest.kt | 7 +- 6 files changed, 14 insertions(+), 545 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt index 6ec62d181e9f7..90328a9e53de8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt @@ -181,7 +181,7 @@ abstract class AbstractBigQueryDestinationAcceptanceTest : DestinationAcceptance val projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText() this._config = config - bigquery = BigQueryDestinationTestUtils.initBigQuery(config, projectId) + bigquery = BigQueryDestinationTestUtils.getBigquery(config) dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt index 10038ad4e3403..6de147d96c8c7 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt @@ -96,7 +96,7 @@ internal class BigQueryDestinationTest { @Throws(IOException::class) protected fun initBigQuery(config: JsonNode) { - bigquery = BigQueryDestinationTestUtils.initBigQuery(config, projectId) + bigquery = BigQueryDestinationTestUtils.initBigQuery(config) try { dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId) } catch (ex: Exception) { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt index 2c3f2590d1083..34276399ddc58 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt @@ -18,7 +18,6 @@ import io.airbyte.integrations.base.destination.typing_deduping.BaseTypingDedupi import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator import io.airbyte.integrations.base.destination.typing_deduping.StreamId import io.airbyte.integrations.destination.bigquery.BigQueryConsts -import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getBigQuery import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId import io.airbyte.protocol.models.v0.* @@ -44,7 +43,7 @@ abstract class AbstractBigQueryTypingDedupingTest : BaseTypingDedupingTest() { val stagingPath = "test_path$uniqueSuffix" val config = BigQueryDestinationTestUtils.createConfig(Path.of(configPath), datasetId, stagingPath) - bq = getBigQuery(config!!) + bq = BigQueryDestinationTestUtils.initBigQuery(config!!) return config } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt index bedbd4e414bb7..bd3d538c2af90 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt @@ -3,535 +3,10 @@ */ package io.airbyte.integrations.destination.bigquery -import com.fasterxml.jackson.databind.JsonNode -import com.google.auth.oauth2.GoogleCredentials -import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.BigQueryException -import com.google.cloud.bigquery.BigQueryOptions import io.airbyte.cdk.AirbyteDestinationRunner import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addThrowableForDeinterpolation -import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.* import io.airbyte.cdk.load.command.aws.AwsToolkitConstants -import io.airbyte.commons.json.Jsons.serialize -import io.airbyte.integrations.destination.bigquery.BigQueryConsts as bqConstants -import io.github.oshai.kotlinlogging.KotlinLogging -import java.io.ByteArrayInputStream -import java.io.IOException -import java.nio.charset.StandardCharsets -import java.util.* - -private val log = KotlinLogging.logger {} - -class BigQueryDestination /*: BaseConnector(), Destination */ { - // - // override fun check(config: JsonNode): AirbyteConnectionStatus? { - // - // try { - // - // val datasetId = BigQueryUtils.getDatasetId(config) - // val datasetLocation = BigQueryUtils.getDatasetLocation(config) - // val bigquery = getBigQuery(config) - // val uploadingMethod = BigQueryUtils.getLoadingMethod(config) - // - // val dataset = BigQueryUtils.getOrCreateDataset(bigquery, datasetId, - // datasetLocation) - // if (dataset.location != datasetLocation) { - // throw ConfigErrorException( - // "Actual dataset location doesn't match to location from config", - // ) - // } - // - // val queryConfig = - // QueryJobConfiguration.newBuilder( - // String.format( - // "SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES` LIMIT 1;", - // datasetId, - // ), - // ) - // .setUseLegacySql(false) - // .build() - // - // val result = BigQueryUtils.executeQuery(bigquery, queryConfig) - // - // if (result.getLeft() == null) { - // - // return AirbyteConnectionStatus() - // .withStatus(AirbyteConnectionStatus.Status.FAILED) - // .withMessage(result.right) - // } - // - // if (UploadingMethod.GCS == uploadingMethod) { - // - // val gcsStatus = checkGcsAccessPermission(config) - // if (gcsStatus!!.status != AirbyteConnectionStatus.Status.SUCCEEDED) { - // return gcsStatus - // } - // - // // Copy a temporary dataset to confirm copy permissions are working - // val bigQueryStatus = checkBigQueryCopyPermission(config) - // - // if (bigQueryStatus.status != AirbyteConnectionStatus.Status.SUCCEEDED) { - // return bigQueryStatus - // } - // } - // - // return - // AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) - // } catch (e: Exception) { - // log.error(e) { "Check failed." } - // throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) - // } - // } - // - // /** - // * This method performs a copy operation to copy data into a temporary table on BigQuery - // to - // * check if the existing permissions are sufficient to copy data. If the permissions are - // not - // * sufficient, then an exception is thrown with a message showing the missing permission - // */ - // private fun checkBigQueryCopyPermission(config: JsonNode): AirbyteConnectionStatus { - // - // // TODO: Need to add a step in this method to first check permissions - // // using testIamPermissions before trying the actual copying of data - // // Created issue for tracking: - // // https://github.com/airbytehq/airbyte-internal-issues/issues/8888 - // - // val datasetLocation = BigQueryUtils.getDatasetLocation(config) - // val bigquery = getBigQuery(config) - // - // val gcsNameTransformer = GcsNameTransformer() - // val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) - // val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) - // val gcsOperations = - // GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) - // - // val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() - // val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) - // val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) - // val defaultDataset = BigQueryUtils.getDatasetId(config) - // - // val finalTableName = - // "_airbyte_bigquery_connection_test_" + - // UUID.randomUUID().toString().replace("-".toRegex(), "") - // - // val rawDatasetOverride: String = - // if (getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).isPresent) { - // getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).get() - // } else { - // JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE - // } - // - // val streamId = - // sqlGenerator.buildStreamId(defaultDataset, finalTableName, rawDatasetOverride) - // - // try { - // - // // Copy a dataset into a BigQuery table to confirm the copy operation is working - // // correctly with the existing permissions - // - // val streamConfig = - // StreamConfig( - // id = streamId, - // postImportAction = ImportType.APPEND, - // primaryKey = listOf(), - // cursor = Optional.empty(), - // columns = linkedMapOf(), - // generationId = 1, - // minimumGenerationId = 1, - // syncId = 0 - // ) - // - // // None of the fields in destination initial status matter - // // for a dummy sync with type-dedupe disabled. We only look at these - // // when we perform final table related setup operations. - // // We just need the streamId to perform the calls in streamOperation. - // - // val initialStatus = - // DestinationInitialStatus( - // streamConfig = streamConfig, - // isFinalTablePresent = false, - // initialRawTableStatus = - // InitialRawTableStatus( - // rawTableExists = false, - // hasUnprocessedRecords = true, - // maxProcessedTimestamp = Optional.empty() - // ), - // initialTempRawTableStatus = - // InitialRawTableStatus( - // rawTableExists = false, - // hasUnprocessedRecords = true, - // maxProcessedTimestamp = Optional.empty() - // ), - // isSchemaMismatch = true, - // isFinalTableEmpty = true, - // destinationState = BigQueryDestinationState(needsSoftReset = false), - // finalTempTableGenerationId = null, - // finalTableGenerationId = null, - // ) - // - // // We simulate a mini-sync to see the raw table code path is exercised. and - // disable T+D - // destinationHandler.createNamespaces(setOf(defaultDataset, rawDatasetOverride)) - // - // val bigQueryGcsStorageOperations = - // BigQueryGcsStorageOperation( - // gcsOperations, - // gcsConfig, - // gcsNameTransformer, - // keepStagingFiles, - // bigquery, - // sqlGenerator, - // destinationHandler, - // ) - // - // val streamOperation: StagingStreamOperations = - // StagingStreamOperations( - // bigQueryGcsStorageOperations, - // initialStatus, - // FileUploadFormat.CSV, - // V2_WITH_GENERATION, - // disableTypeDedupe = true - // ) - // - // // Dummy message - // val data = - // """ - // {"testKey": "testValue"} - // """.trimIndent() - // - // val message = - // PartialAirbyteMessage() - // .withSerialized(data) - // .withRecord( - // PartialAirbyteRecordMessage() - // .withEmittedAt(System.currentTimeMillis()) - // .withMeta( - // AirbyteRecordMessageMeta(), - // ), - // ) - // - // streamOperation.writeRecords(streamConfig, listOf(message).stream()) - // streamOperation.finalizeTable( - // streamConfig, - // StreamSyncSummary(1, - // AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE), - // ) - // - // // Note: Deletion of the temporary table is being done in the finally block below - // - // return - // AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) - // } catch (e: Exception) { - // - // log.error(e) { "checkGcsCopyPermission failed." } - // - // throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!) - // } finally { - // - // try { - // // In the finally block, clean up the raw table - // // If there was an exception in the flow above, then the table may still exist - // destinationHandler.execute( - // Sql.of( - // "DROP TABLE IF EXISTS - // $projectId.${streamId.rawNamespace}.${streamId.rawName};", - // ), - // ) - // } catch (e: Exception) { - // log.error(e) { - // "Error while cleaning up the temporary table... not throwing a new - // exception" - // } - // } - // } - // } - // - // /** - // * This method does two checks: 1) permissions related to the bucket, and 2) the ability - // to - // * create and delete an actual file. The latter is important because even if the service - // account - // * may have the proper permissions, the HMAC keys can only be verified by running the - // actual GCS - // * check. - // */ - // private fun checkGcsAccessPermission(config: JsonNode): AirbyteConnectionStatus? { - // val loadingMethod = config[bqConstants.LOADING_METHOD] - // val bucketName = loadingMethod[bqConstants.GCS_BUCKET_NAME].asText() - // val missingPermissions: MutableList = ArrayList() - // - // try { - // val credentials = getServiceAccountCredentials(config) - // val storage: Storage = - // StorageOptions.newBuilder() - // .setProjectId(config[bqConstants.CONFIG_PROJECT_ID].asText()) - // .setCredentials(credentials) - // .setHeaderProvider(BigQueryUtils.headerProvider) - // .build() - // .service - // val permissionsCheckStatusList: List = - // storage.testIamPermissions(bucketName, REQUIRED_GCS_PERMISSIONS) - // - // // testIamPermissions returns a list of booleans - // // in the same order of the presented permissions list - // missingPermissions.addAll( - // permissionsCheckStatusList - // .asSequence() - // .withIndex() - // .filter { !it.value } - // .map { REQUIRED_GCS_PERMISSIONS[it.index] } - // .toList(), - // ) - // - // val gcsDestination: BaseGcsDestination = object : BaseGcsDestination() {} - // val gcsJsonNodeConfig = BigQueryUtils.getGcsJsonNodeConfig(config) - // return gcsDestination.check(gcsJsonNodeConfig) - // } catch (e: Exception) { - // val message = StringBuilder("Cannot access the GCS bucket.") - // if (!missingPermissions.isEmpty()) { - // message - // .append(" The following permissions are missing on the service account: ") - // .append(java.lang.String.join(", ", missingPermissions)) - // .append(".") - // } - // message.append( - // " Please make sure the service account can access the bucket path, and the - // HMAC keys are correct.", - // ) - // - // log.error(e) { message.toString() } - // throw ConfigErrorException( - // "Could not access the GCS bucket with the provided configuration.\n", - // e, - // ) - // } - // } - // - // /** - // * Returns a [AirbyteMessageConsumer] based on whether the uploading mode is STANDARD - // INSERTS or - // * using STAGING - // * - // * @param config - // * - integration-specific configuration object as json. e.g. { "username": "airbyte", - // * "password": "super secure" } - // * @param catalog - // * - schema of the incoming messages. - // */ - // override fun getConsumer( - // config: JsonNode, - // catalog: ConfiguredAirbyteCatalog, - // outputRecordCollector: Consumer - // ): AirbyteMessageConsumer? { - // throw UnsupportedOperationException("Should use getSerializedMessageConsumer") - // } - // - // @Throws(Exception::class) - // override fun getSerializedMessageConsumer( - // config: JsonNode, - // catalog: ConfiguredAirbyteCatalog, - // outputRecordCollector: Consumer - // ): SerializedAirbyteMessageConsumer { - // val uploadingMethod = BigQueryUtils.getLoadingMethod(config) - // val defaultNamespace = BigQueryUtils.getDatasetId(config) - // val disableTypeDedupe = BigQueryUtils.getDisableTypeDedupFlag(config) - // val datasetLocation = BigQueryUtils.getDatasetLocation(config) - // val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() - // val bigquery = getBigQuery(config) - // val rawNamespaceOverride = getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET) - // - // addAllStringsInConfigForDeinterpolation(config) - // val serviceAccountKey = config[bqConstants.CONFIG_CREDS] - // if (serviceAccountKey != null) { - // // If the service account key is a non-null string, we will try to - // // deserialize it. Otherwise, we will let the Google library find it in - // // the environment during the client initialization. - // if (serviceAccountKey.isTextual) { - // // There are cases where we fail to deserialize the service account key. In - // these - // // cases, we - // // shouldn't do anything. - // // Google's creds library is more lenient with JSON-parsing than Jackson, and - // I'd - // // rather just let it - // // go. - // tryDeserialize(serviceAccountKey.asText()).ifPresent { obj: JsonNode -> - // addAllStringsInConfigForDeinterpolation(obj) - // } - // } else { - // addAllStringsInConfigForDeinterpolation(serviceAccountKey) - // } - // } - // - // val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation) - // val parsedCatalog = - // parseCatalog( - // sqlGenerator, - // defaultNamespace, - // - // rawNamespaceOverride.orElse(JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE), - // catalog, - // ) - // val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation) - // - // val migrations = - // listOf( - // BigQueryDV2Migration(sqlGenerator, bigquery), - // BigqueryAirbyteMetaAndGenerationIdMigration(bigquery), - // ) - // - // if (uploadingMethod == UploadingMethod.STANDARD) { - // val bigQueryClientChunkSize = BigQueryUtils.getBigQueryClientChunkSize(config) - // val bigQueryLoadingStorageOperation = - // BigQueryDirectLoadingStorageOperation( - // bigquery, - // bigQueryClientChunkSize, - // BigQueryRecordFormatter(), - // sqlGenerator, - // destinationHandler, - // datasetLocation, - // ) - // val syncOperation = - // DefaultSyncOperation( - // parsedCatalog, - // destinationHandler, - // defaultNamespace, - // { initialStatus: DestinationInitialStatus, - // disableTD - // -> - // StandardStreamOperation( - // bigQueryLoadingStorageOperation, - // initialStatus, - // disableTD - // ) - // }, - // migrations, - // disableTypeDedupe, - // ) - // return createDirectUploadConsumer( - // outputRecordCollector, - // syncOperation, - // catalog, - // defaultNamespace, - // ) - // } - // - // val gcsNameTransformer = GcsNameTransformer() - // val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config) - // val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config) - // val gcsOperations = - // GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig) - // - // val bigQueryGcsStorageOperations = - // BigQueryGcsStorageOperation( - // gcsOperations, - // gcsConfig, - // gcsNameTransformer, - // keepStagingFiles, - // bigquery, - // sqlGenerator, - // destinationHandler, - // ) - // val syncOperation: SyncOperation = - // DefaultSyncOperation( - // parsedCatalog, - // destinationHandler, - // defaultNamespace, - // { initialStatus: DestinationInitialStatus, disableTD - // -> - // StagingStreamOperations( - // bigQueryGcsStorageOperations, - // initialStatus, - // FileUploadFormat.CSV, - // V2_WITH_GENERATION, - // disableTD - // ) - // }, - // migrations, - // disableTypeDedupe, - // ) - // return createStagingConsumer( - // outputRecordCollector, - // syncOperation, - // catalog, - // defaultNamespace, - // ) - // } - // - // private fun parseCatalog( - // sqlGenerator: BigQuerySqlGenerator, - // defaultNamespace: String, - // rawNamespaceOverride: String, - // catalog: ConfiguredAirbyteCatalog - // ): ParsedCatalog { - // val catalogParser = - // CatalogParser( - // sqlGenerator, - // defaultNamespace = defaultNamespace, - // rawNamespace = rawNamespaceOverride, - // ) - // - // return catalogParser.parseCatalog(catalog) - // } - // - // override val isV2Destination: Boolean - // get() = true - // - companion object { - - private val REQUIRED_GCS_PERMISSIONS = - listOf( - "storage.multipartUploads.abort", - "storage.multipartUploads.create", - "storage.objects.create", - "storage.objects.delete", - "storage.objects.get", - "storage.objects.list", - ) - - @JvmStatic - fun getBigQuery(config: JsonNode): BigQuery { - val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText() - - try { - val bigQueryBuilder = BigQueryOptions.newBuilder() - val credentials = getServiceAccountCredentials(config) - return bigQueryBuilder - .setProjectId(projectId) - .setCredentials(credentials) - .setHeaderProvider(BigQueryUtils.headerProvider) - .build() - .service - } catch (e: IOException) { - throw RuntimeException(e) - } - } - - @JvmStatic - @Throws(IOException::class) - fun getServiceAccountCredentials(config: JsonNode): GoogleCredentials { - val serviceAccountKey = config[bqConstants.CONFIG_CREDS] - // Follows this order of resolution: - // https://cloud.google.com/java/docs/reference/google-auth-library/latest/com.google.auth.oauth2.GoogleCredentials#com_google_auth_oauth2_GoogleCredentials_getApplicationDefault - if (serviceAccountKey == null) { - log.info { - "No service account key json is provided. It is required if you are using Airbyte cloud." - } - log.info { "Using the default service account credential from environment." } - return GoogleCredentials.getApplicationDefault() - } - - // The JSON credential can either be a raw JSON object, or a serialized JSON object. - val credentialsString = - if (serviceAccountKey.isObject) serialize(serviceAccountKey) - else serviceAccountKey.asText() - return GoogleCredentials.fromStream( - ByteArrayInputStream(credentialsString.toByteArray(StandardCharsets.UTF_8)), - ) - } - } -} val additionalMicronautEnvs = listOf(AwsToolkitConstants.MICRONAUT_ENVIRONMENT) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.kt b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.kt index 776d05f10f963..afd638c4c4a2b 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTestUtils.kt @@ -8,12 +8,14 @@ import com.amazonaws.services.s3.model.DeleteObjectsRequest import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.BigQueryOptions import com.google.cloud.bigquery.Dataset import com.google.cloud.bigquery.DatasetInfo import io.airbyte.cdk.load.util.Jsons -import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getServiceAccountCredentials import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getLoadingMethod +import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory +import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification +import io.airbyte.integrations.destination.bigquery.util.BigqueryClientFactory +import io.github.oshai.kotlinlogging.KotlinLogging import java.io.IOException import java.nio.file.Files import java.nio.file.Path @@ -21,6 +23,8 @@ import java.util.* import org.slf4j.Logger import org.slf4j.LoggerFactory +private val logger = KotlinLogging.logger {} + object BigQueryDestinationTestUtils { private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDestinationTestUtils::class.java) @@ -84,18 +88,14 @@ object BigQueryDestinationTestUtils { * for cleaning up BigQuery dataset after the test * * @param config - * @param projectId * @return * @throws IOException */ @Throws(IOException::class) - fun initBigQuery(config: JsonNode, projectId: String?): BigQuery { - val credentials = getServiceAccountCredentials(config) - return BigQueryOptions.newBuilder() - .setProjectId(projectId) - .setCredentials(credentials) - .build() - .service + fun initBigQuery(config: JsonNode): BigQuery { + val spec = Jsons.treeToValue(config, BigquerySpecification::class.java) + val parsedConfig = BigqueryConfigurationFactory().make(spec) + return BigqueryClientFactory(parsedConfig).make() } /** diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt index 9a07f16dfa8bb..1d766f69834a9 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/test-integration/kotlin/io/airbyte/integrations/destination/bigquery/BigqueryWriteTest.kt @@ -15,10 +15,7 @@ import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification import java.nio.file.Path import org.junit.jupiter.api.Test -abstract class BigqueryWriteTest( - configContents: String, - additionalMicronautEnvs: List = emptyList() -) : +abstract class BigqueryWriteTest(configContents: String) : BasicFunctionalityIntegrationTest( configContents = configContents, BigquerySpecification::class.java, @@ -85,8 +82,6 @@ class GcsRawOverrideDisableTd : stagingPath = "test_path/$DEFAULT_NAMESPACE_PLACEHOLDER", ) .serializeToString(), - // TODO make a base BigqueryGcsWriteTest class to handle this - additionalMicronautEnvs = additionalMicronautEnvs, ) { @Test override fun testBasicWrite() { From b47374dda53aae6e0a642649e3a301feb8c53b71 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 17 Apr 2025 15:08:53 -0700 Subject: [PATCH 23/55] make legacy tests mostly work --- ...stractBigQueryDestinationAcceptanceTest.kt | 7 +- .../bigquery/BigQueryDestinationTest.kt | 178 +---- .../AbstractBigQueryTypingDedupingTest.kt | 3 +- .../BigQuerySqlGeneratorIntegrationTest.kt | 643 ++-------------- .../LegacyBigQuerySqlGenerator.kt | 689 ++++++++++++++++++ 5 files changed, 745 insertions(+), 775 deletions(-) create mode 100644 airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt index 90328a9e53de8..4aaa49582d4f5 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt @@ -19,7 +19,7 @@ import io.airbyte.commons.json.Jsons.deserialize import io.airbyte.commons.string.Strings.addRandomSuffix import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator +import io.airbyte.integrations.destination.bigquery.typing_deduping.LegacyBigQuerySqlGenerator import java.io.IOException import java.nio.file.Path import java.util.* @@ -123,7 +123,7 @@ abstract class AbstractBigQueryDestinationAcceptanceTest : DestinationAcceptance streamSchema: JsonNode ): List { val streamId = - BigQuerySqlGenerator(null, null) + LegacyBigQuerySqlGenerator(null, null) .buildStreamId( namespace, streamName, @@ -179,9 +179,8 @@ abstract class AbstractBigQueryDestinationAcceptanceTest : DestinationAcceptance val config = BigQueryDestinationTestUtils.createConfig(secretsFile, datasetId, stagingPathSuffix) - val projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText() this._config = config - bigquery = BigQueryDestinationTestUtils.getBigquery(config) + bigquery = BigQueryDestinationTestUtils.initBigQuery(config) dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt index 6de147d96c8c7..5695ac7942f17 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt @@ -8,22 +8,18 @@ import com.fasterxml.jackson.databind.JsonNode import com.google.cloud.bigquery.* import com.google.common.collect.ImmutableMap import com.google.common.collect.Lists -import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer -import io.airbyte.cdk.integrations.base.Destination import io.airbyte.cdk.integrations.base.DestinationConfig import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.destination.NamingConventionTransformer import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig import io.airbyte.commons.json.Jsons.deserialize import io.airbyte.commons.json.Jsons.jsonNode -import io.airbyte.commons.resources.MoreResources.readResource import io.airbyte.commons.string.Strings.addRandomSuffix import io.airbyte.integrations.destination.bigquery.BigQueryUtils.createPartitionedTableIfNotExists import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetLocation import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getGcsJsonNodeConfig import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getOrCreateDataset -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator import io.airbyte.protocol.models.Field import io.airbyte.protocol.models.JsonSchemaType import io.airbyte.protocol.models.v0.* @@ -36,12 +32,10 @@ import java.util.function.Function import java.util.stream.Collectors import java.util.stream.Stream import java.util.stream.StreamSupport -import org.assertj.core.api.Assertions import org.junit.jupiter.api.* import org.junit.jupiter.params.ParameterizedTest import org.junit.jupiter.params.provider.Arguments import org.junit.jupiter.params.provider.MethodSource -import org.mockito.Mockito import org.mockito.Mockito.mock import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -126,86 +120,24 @@ internal class BigQueryDestinationTest { BigQueryDestinationTestUtils.tearDownGcs(s3Client, config, LOGGER) } - @Test - @Throws(Exception::class) - fun testSpec() { - val actual = BigQueryDestination().spec() - val resourceString = readResource("spec.json") - val expected = deserialize(resourceString, ConnectorSpecification::class.java) - - org.junit.jupiter.api.Assertions.assertEquals(expected, actual) - } + @Disabled @Test @Throws(Exception::class) fun testSpec() {} + @Disabled @ParameterizedTest @MethodSource("successTestConfigProvider") @Throws(IOException::class) - fun testCheckSuccess(configName: String) { - val testConfig = configs!![configName] - val actual = BigQueryDestination().check(testConfig!!) - val expected = - AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED) - org.junit.jupiter.api.Assertions.assertEquals(expected, actual) - } + fun testCheckSuccess(configName: String) {} + @Disabled @ParameterizedTest @MethodSource("failCheckTestConfigProvider") - fun testCheckFailures(configName: String, error: String?) { - // TODO: this should always throw ConfigErrorException - val testConfig = configs!![configName] - - val ex = - org.junit.jupiter.api.Assertions.assertThrows(Exception::class.java) { - BigQueryDestination().check(testConfig!!) - } - Assertions.assertThat(ex.message).contains(error) - } + fun testCheckFailures(configName: String, error: String?) {} @Disabled @ParameterizedTest @MethodSource("successTestConfigProvider") @Throws(Exception::class) - fun testWriteSuccess(configName: String) { - initBigQuery(config) - val testConfig = configs!![configName] - val destination = BigQueryDestination() - val consumer = - destination.getConsumer(testConfig!!, catalog!!) { message: AirbyteMessage? -> - Destination.defaultOutputRecordCollector(message) - } - - consumer!!.start() - consumer.accept(MESSAGE_USERS1) - consumer.accept(MESSAGE_TASKS1) - consumer.accept(MESSAGE_USERS2) - consumer.accept(MESSAGE_TASKS2) - consumer.accept(MESSAGE_STATE) - consumer.close() - - val usersActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(USERS_STREAM_NAME)) - val expectedUsersJson: List = - Lists.newArrayList(MESSAGE_USERS1.record.data, MESSAGE_USERS2.record.data) - org.junit.jupiter.api.Assertions.assertEquals(expectedUsersJson.size, usersActual.size) - org.junit.jupiter.api.Assertions.assertTrue( - expectedUsersJson.containsAll(usersActual) && usersActual.containsAll(expectedUsersJson) - ) - - val tasksActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(TASKS_STREAM_NAME)) - val expectedTasksJson: List = - Lists.newArrayList(MESSAGE_TASKS1.record.data, MESSAGE_TASKS2.record.data) - org.junit.jupiter.api.Assertions.assertEquals(expectedTasksJson.size, tasksActual.size) - org.junit.jupiter.api.Assertions.assertTrue( - expectedTasksJson.containsAll(tasksActual) && tasksActual.containsAll(expectedTasksJson) - ) - - assertTmpTablesNotPresent( - catalog!! - .streams - .stream() - .map { obj: ConfiguredAirbyteStream -> obj.stream } - .map { obj: AirbyteStream -> obj.name } - .collect(Collectors.toList()) - ) - } + fun testWriteSuccess(configName: String) {} @Test @Throws(Exception::class) @@ -248,44 +180,7 @@ internal class BigQueryDestinationTest { @ParameterizedTest @MethodSource("failWriteTestConfigProvider") @Throws(Exception::class) - fun testWriteFailure(configName: String, error: String?) { - initBigQuery(config) - val testConfig = configs!![configName] - val ex = - org.junit.jupiter.api.Assertions.assertThrows(Exception::class.java) { - val consumer = - Mockito.spy( - BigQueryDestination().getConsumer(testConfig!!, catalog!!) { - message: AirbyteMessage? -> - Destination.defaultOutputRecordCollector(message) - } - ) - consumer!!.start() - } - Assertions.assertThat(ex.message).contains(error) - - val tableNames = - catalog!! - .streams - .stream() - .map { obj: ConfiguredAirbyteStream -> obj.stream } - .map { obj: AirbyteStream -> obj.name } - .toList() - assertTmpTablesNotPresent( - catalog!! - .streams - .stream() - .map { obj: ConfiguredAirbyteStream -> obj.stream } - .map { obj: AirbyteStream -> obj.name } - .collect(Collectors.toList()) - ) - // assert that no tables were created. - org.junit.jupiter.api.Assertions.assertTrue( - fetchNamesOfTablesInDb().stream().noneMatch { tableName: String -> - tableNames.stream().anyMatch { prefix: String -> tableName.startsWith(prefix) } - } - ) - } + fun testWriteFailure(configName: String, error: String?) {} @Throws(InterruptedException::class) private fun fetchNamesOfTablesInDb(): Set { @@ -372,64 +267,7 @@ internal class BigQueryDestinationTest { @ParameterizedTest @MethodSource("successTestConfigProviderBase") @Throws(Exception::class) - fun testWritePartitionOverUnpartitioned(configName: String) { - val testConfig = configs!![configName] - initBigQuery(config) - val streamId = - BigQuerySqlGenerator(projectId, null) - .buildStreamId( - datasetId!!, - USERS_STREAM_NAME, - JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE - ) - val dataset = - BigQueryDestinationTestUtils.initDataSet(config, bigquery, streamId.rawNamespace) - createUnpartitionedTable(bigquery!!, dataset, streamId.rawName) - org.junit.jupiter.api.Assertions.assertFalse( - isTablePartitioned(bigquery!!, dataset, streamId.rawName) - ) - val destination = BigQueryDestination() - val consumer = - destination.getConsumer(testConfig!!, catalog!!) { message: AirbyteMessage? -> - Destination.defaultOutputRecordCollector(message) - } - - consumer!!.start() - consumer.accept(MESSAGE_USERS1) - consumer.accept(MESSAGE_TASKS1) - consumer.accept(MESSAGE_USERS2) - consumer.accept(MESSAGE_TASKS2) - consumer.accept(MESSAGE_STATE) - consumer.close() - - val usersActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(USERS_STREAM_NAME)) - val expectedUsersJson: List = - Lists.newArrayList(MESSAGE_USERS1.record.data, MESSAGE_USERS2.record.data) - org.junit.jupiter.api.Assertions.assertEquals(expectedUsersJson.size, usersActual.size) - org.junit.jupiter.api.Assertions.assertTrue( - expectedUsersJson.containsAll(usersActual) && usersActual.containsAll(expectedUsersJson) - ) - - val tasksActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(TASKS_STREAM_NAME)) - val expectedTasksJson: List = - Lists.newArrayList(MESSAGE_TASKS1.record.data, MESSAGE_TASKS2.record.data) - org.junit.jupiter.api.Assertions.assertEquals(expectedTasksJson.size, tasksActual.size) - org.junit.jupiter.api.Assertions.assertTrue( - expectedTasksJson.containsAll(tasksActual) && tasksActual.containsAll(expectedTasksJson) - ) - - assertTmpTablesNotPresent( - catalog!! - .streams - .stream() - .map { obj: ConfiguredAirbyteStream -> obj.stream } - .map { obj: AirbyteStream -> obj.name } - .collect(Collectors.toList()) - ) - org.junit.jupiter.api.Assertions.assertTrue( - isTablePartitioned(bigquery!!, dataset, streamId.rawName) - ) - } + fun testWritePartitionOverUnpartitioned(configName: String) {} private fun createUnpartitionedTable( bigquery: BigQuery?, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt index 34276399ddc58..b4f971059b46b 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/AbstractBigQueryTypingDedupingTest.kt @@ -98,7 +98,8 @@ abstract class AbstractBigQueryTypingDedupingTest : BaseTypingDedupingTest() { } override val sqlGenerator: SqlGenerator - get() = BigQuerySqlGenerator(config!![BigQueryConsts.CONFIG_PROJECT_ID].asText(), null) + get() = + LegacyBigQuerySqlGenerator(config!![BigQueryConsts.CONFIG_PROJECT_ID].asText(), null) @Test @Throws(Exception::class) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.kt index eba281e075bd8..8b45f0c0297b9 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGeneratorIntegrationTest.kt @@ -1,620 +1,63 @@ /* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ + package io.airbyte.integrations.destination.bigquery.typing_deduping import com.fasterxml.jackson.databind.JsonNode import com.fasterxml.jackson.databind.node.ObjectNode -import com.google.cloud.bigquery.* -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.commons.json.Jsons.deserialize +import com.google.cloud.bigquery.FieldValueList +import com.google.cloud.bigquery.Schema +import com.google.cloud.bigquery.StandardSQLTypeName +import com.google.cloud.bigquery.TableResult import io.airbyte.commons.json.Jsons.deserializeExact import io.airbyte.commons.json.Jsons.emptyObject import io.airbyte.commons.json.Jsons.jsonNode -import io.airbyte.integrations.base.destination.typing_deduping.* -import io.airbyte.integrations.destination.bigquery.BigQueryConsts -import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getBigQuery -import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState -import java.nio.file.Files -import java.nio.file.Path -import java.time.Duration -import java.util.* -import java.util.Map -import java.util.stream.Collectors -import kotlin.collections.LinkedHashMap -import kotlin.collections.List -import kotlin.collections.emptyList -import kotlin.collections.indices -import org.apache.commons.text.StringSubstitutor -import org.junit.jupiter.api.Assertions -import org.junit.jupiter.api.BeforeAll -import org.junit.jupiter.api.Disabled -import org.junit.jupiter.api.Test -import org.junit.jupiter.api.parallel.Execution -import org.junit.jupiter.api.parallel.ExecutionMode -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.ValueSource -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -@Execution(ExecutionMode.CONCURRENT) -class BigQuerySqlGeneratorIntegrationTest : - BaseSqlGeneratorIntegrationTest() { - override val sqlGenerator: BigQuerySqlGenerator - get() = BigQuerySqlGenerator(projectId, datasetLocation) - - override val destinationHandler: BigQueryDestinationHandler - get() = BigQueryDestinationHandler(bq!!, "US") - - override fun createNamespace(namespace: String) { - bq!!.create( - DatasetInfo.newBuilder( - namespace - ) // This unfortunately doesn't delete the actual dataset after 3 days, but at least - // we'll clear out - // old tables automatically - .setDefaultTableLifetime(Duration.ofDays(3).toMillis()) - .build() - ) - } - - @Throws(InterruptedException::class) - override fun createRawTable(streamId: StreamId) { - bq!!.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - Map.of("raw_table_id", streamId.rawTableId(BigQuerySqlGenerator.QUOTE)) - ) - .replace( - """ - CREATE TABLE ${'$'}{raw_table_id} ( - _airbyte_raw_id STRING NOT NULL, - _airbyte_data STRING NOT NULL, - _airbyte_extracted_at TIMESTAMP NOT NULL, - _airbyte_loaded_at TIMESTAMP, - _airbyte_meta STRING, - _airbyte_generation_id INTEGER - ) PARTITION BY ( - DATE_TRUNC(_airbyte_extracted_at, DAY) - ) CLUSTER BY _airbyte_loaded_at; - - """.trimIndent() - ) - ) - .build() - ) - } - - @Throws(Exception::class) - override fun createV1RawTable(v1RawTable: StreamId) { - bq!!.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - Map.of( - "raw_table_id", - v1RawTable.rawTableId(BigQuerySqlGenerator.QUOTE) - ) - ) - .replace( - """ - CREATE TABLE ${'$'}{raw_table_id} ( - _airbyte_ab_id STRING NOT NULL, - _airbyte_data STRING NOT NULL, - _airbyte_emitted_at TIMESTAMP NOT NULL, - ) PARTITION BY ( - DATE_TRUNC(_airbyte_emitted_at, DAY) - ) CLUSTER BY _airbyte_emitted_at; - - """.trimIndent() - ) - ) - .build() - ) - } - - @Throws(InterruptedException::class) - @SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE") - override fun insertFinalTableRecords( - includeCdcDeletedAt: Boolean, - streamId: StreamId, - suffix: String?, - records: List, - generationId: Long - ) { - val columnNames = - if (includeCdcDeletedAt) FINAL_TABLE_COLUMN_NAMES_CDC else FINAL_TABLE_COLUMN_NAMES - val cdcDeletedAtDecl = if (includeCdcDeletedAt) ",`_ab_cdc_deleted_at` TIMESTAMP" else "" - val cdcDeletedAtName = if (includeCdcDeletedAt) ",`_ab_cdc_deleted_at`" else "" - val recordsText = - records - .stream() // For each record, convert it to a string like "(rawId, extractedAt, - // loadedAt, data)" - .map { record: JsonNode -> - columnNames - .stream() - .map { fieldName: String? -> record[fieldName] } - .map { r: JsonNode? -> - if (r == null) { - return@map "NULL" - } - val stringContents = - if (r.isTextual) { - r.asText() - } else { - r.toString() - } - '"'.toString() + - stringContents // Serialized json might contain backslashes and - // double quotes. Escape them. - .replace("\\", "\\\\") - .replace("\"", "\\\"") + - '"' - } - .collect(Collectors.joining(",")) - } - .map { row: String -> "($row)" } - .collect(Collectors.joining(",")) - - bq!!.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - Map.of( - "final_table_id", - streamId.finalTableId(BigQuerySqlGenerator.QUOTE, suffix!!), - "cdc_deleted_at_name", - cdcDeletedAtName, - "cdc_deleted_at_decl", - cdcDeletedAtDecl, - "records", - recordsText - ) - ) - .replace( // Similar to insertRawTableRecords, some of these columns are - // declared as string and wrapped in - // parse_json(). - // There's also a bunch of casting, because bigquery doesn't coerce - // strings to e.g. int - """ - insert into ${'$'}{final_table_id} ( - _airbyte_raw_id, - _airbyte_extracted_at, - _airbyte_meta, - _airbyte_generation_id, - `id1`, - `id2`, - `updated_at`, - `struct`, - `array`, - `string`, - `number`, - `integer`, - `boolean`, - `timestamp_with_timezone`, - `timestamp_without_timezone`, - `time_with_timezone`, - `time_without_timezone`, - `date`, - `unknown` - ${'$'}{cdc_deleted_at_name} - ) - select - _airbyte_raw_id, - _airbyte_extracted_at, - parse_json(_airbyte_meta), - _airbyte_generation_id, - cast(`id1` as int64), - cast(`id2` as int64), - `updated_at`, - parse_json(`struct`), - parse_json(`array`), - `string`, - cast(`number` as numeric), - cast(`integer` as int64), - cast(`boolean` as boolean), - `timestamp_with_timezone`, - `timestamp_without_timezone`, - `time_with_timezone`, - `time_without_timezone`, - `date`, - parse_json(`unknown`) - ${'$'}{cdc_deleted_at_name} - from unnest([ - STRUCT< - _airbyte_raw_id STRING, - _airbyte_extracted_at TIMESTAMP, - _airbyte_meta STRING, - _airbyte_generation_id INTEGER, - `id1` STRING, - `id2` STRING, - `updated_at` TIMESTAMP, - `struct` STRING, - `array` STRING, - `string` STRING, - `number` STRING, - `integer` STRING, - `boolean` STRING, - `timestamp_with_timezone` TIMESTAMP, - `timestamp_without_timezone` DATETIME, - `time_with_timezone` STRING, - `time_without_timezone` TIME, - `date` DATE, - `unknown` STRING - ${'$'}{cdc_deleted_at_decl} - > - ${'$'}{records} - ]) - - """.trimIndent() - ) - ) - .build() - ) - } - - private fun stringifyRecords(records: List, columnNames: List): String { - return records - .stream() // For each record, convert it to a string like "(rawId, extractedAt, - // loadedAt, data)" - .map { record: JsonNode -> - columnNames - .stream() - .map { fieldName: String? -> record[fieldName] } - .map { r: JsonNode? -> - if (r == null) { - return@map "NULL" - } - val stringContents = - if (r.isTextual) { - r.asText() - } else { - r.toString() - } - '"'.toString() + - stringContents // Serialized json might contain backslashes and double - // quotes. Escape them. - .replace("\\", "\\\\") - .replace("\"", "\\\"") + - '"' - } - .collect(Collectors.joining(",")) - } - .map { row: String -> "($row)" } - .collect(Collectors.joining(",")) - } - - @Throws(InterruptedException::class) - override fun insertRawTableRecords(streamId: StreamId, records: List) { - val recordsText = - stringifyRecords(records, JavaBaseConstants.V2_RAW_TABLE_COLUMN_NAMES_WITH_GENERATION) - - bq!!.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - Map.of( - "raw_table_id", - streamId.rawTableId(BigQuerySqlGenerator.QUOTE), - "records", - recordsText - ) - ) - .replace( // TODO: Perform a normal insert - edward - """ - INSERT INTO ${'$'}{raw_table_id} (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_loaded_at, _airbyte_data, _airbyte_meta, _airbyte_generation_id) - SELECT _airbyte_raw_id, _airbyte_extracted_at, _airbyte_loaded_at, _airbyte_data, _airbyte_meta, cast(_airbyte_generation_id as int64) FROM UNNEST([ - STRUCT<`_airbyte_raw_id` STRING, `_airbyte_extracted_at` TIMESTAMP, `_airbyte_loaded_at` TIMESTAMP, _airbyte_data STRING, _airbyte_meta STRING, `_airbyte_generation_id` STRING> - ${'$'}{records} - ]) - - """.trimIndent() - ) - ) - .build() - ) - } - - @Throws(Exception::class) - override fun insertV1RawTableRecords(streamId: StreamId, records: List) { - val recordsText = stringifyRecords(records, JavaBaseConstants.LEGACY_RAW_TABLE_COLUMNS) - bq!!.query( - QueryJobConfiguration.newBuilder( - StringSubstitutor( - Map.of( - "v1_raw_table_id", - streamId.rawTableId(BigQuerySqlGenerator.QUOTE), - "records", - recordsText - ) - ) - .replace( - """ - INSERT INTO ${'$'}{v1_raw_table_id} (_airbyte_ab_id, _airbyte_data, _airbyte_emitted_at) - SELECT _airbyte_ab_id, _airbyte_data, _airbyte_emitted_at FROM UNNEST([ - STRUCT<`_airbyte_ab_id` STRING, _airbyte_data STRING, `_airbyte_emitted_at` TIMESTAMP> - ${'$'}{records} - ]) - - """.trimIndent() - ) - ) - .build() - ) - } - - @Throws(Exception::class) - override fun dumpRawTableRecords(streamId: StreamId): List { - val result = - bq!!.query( - QueryJobConfiguration.of( - "SELECT * FROM " + streamId.rawTableId(BigQuerySqlGenerator.QUOTE) - ) - ) - return toJsonRecords(result) - } - - @Throws(Exception::class) - @SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE") - override fun dumpFinalTableRecords(streamId: StreamId, suffix: String?): List { - val result = - bq!!.query( - QueryJobConfiguration.of( - "SELECT * FROM " + streamId.finalTableId(BigQuerySqlGenerator.QUOTE, suffix!!) - ) - ) - return toJsonRecords(result) - } - - override fun teardownNamespace(namespace: String) { - bq!!.delete(namespace, BigQuery.DatasetDeleteOption.deleteContents()) - } - - override val supportsSafeCast: Boolean - get() = true - - @Test - @Throws(Exception::class) - override fun testCreateTableIncremental() { - destinationHandler.execute(generator.createTable(incrementalDedupStream, "", false)) - - val table = bq!!.getTable(namespace, "users_final") - // The table should exist - Assertions.assertNotNull(table) - val schema = table.getDefinition().schema - // And we should know exactly what columns it contains - Assertions - .assertEquals( // Would be nice to assert directly against StandardSQLTypeName, but - // bigquery returns schemas of - // LegacySQLTypeName. So we have to translate. - Schema.of( - Field.newBuilder( - "_airbyte_raw_id", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING) - ) - .setMode(Field.Mode.REQUIRED) - .build(), - Field.newBuilder( - "_airbyte_extracted_at", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP) - ) - .setMode(Field.Mode.REQUIRED) - .build(), - Field.newBuilder( - "_airbyte_meta", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON) - ) - .setMode(Field.Mode.REQUIRED) - .build(), - Field.newBuilder( - "_airbyte_generation_id", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64) - ) - .build(), - Field.of("id1", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)), - Field.of("id2", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)), - Field.of( - "updated_at", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP) - ), - Field.of( - "struct", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON) - ), - Field.of( - "array", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON) - ), - Field.of( - "string", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING) - ), - Field.of( - "number", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.NUMERIC) - ), - Field.of( - "integer", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64) - ), - Field.of( - "boolean", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.BOOL) - ), - Field.of( - "timestamp_with_timezone", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP) - ), - Field.of( - "timestamp_without_timezone", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.DATETIME) - ), - Field.of( - "time_with_timezone", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING) - ), - Field.of( - "time_without_timezone", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIME) - ), - Field.of("date", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.DATE)), - Field.of( - "unknown", - LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON) - ) - ), - schema - ) - // TODO this should assert partitioning/clustering configs - } - - @Test - @Throws(InterruptedException::class) - fun testCreateTableInOtherRegion() { - val destinationHandler = BigQueryDestinationHandler(bq!!, "asia-east1") - // We're creating the dataset in the wrong location in the @BeforeEach block. Explicitly - // delete it. - bq!!.getDataset(namespace).delete() - val sqlGenerator = BigQuerySqlGenerator(projectId, "asia-east1") - destinationHandler.execute(sqlGenerator.createSchema(namespace)) - destinationHandler.execute(sqlGenerator.createTable(incrementalDedupStream, "", false)) - - // Empirically, it sometimes takes Bigquery nearly 30 seconds to propagate the dataset's - // existence. - // Give ourselves 2 minutes just in case. - for (i in 0..119) { - val dataset = bq!!.getDataset(DatasetId.of(bq!!.options.projectId, namespace)) - if (dataset == null) { - LOGGER.info("Sleeping and trying again... ({})", i) - Thread.sleep(1000) - } else { - Assertions.assertEquals("asia-east1", dataset.location) - return - } - } - Assertions.fail("Dataset does not exist") - } +object BigQuerySqlGeneratorIntegrationTest { /** - * Bigquery column names aren't allowed to start with certain prefixes. Verify that we throw an - * error in these cases. + * TableResult contains records in a somewhat nonintuitive format (and it avoids loading them + * all into memory). That's annoying for us since we're working with small test data, so just + * pull everything into a list. */ - @ParameterizedTest - @ValueSource( - strings = - ["_table_", "_file_", "_partition_", "_row_timestamp_", "__root__", "_colidentifier_"] - ) - fun testFailureOnReservedColumnNamePrefix(prefix: String) { - val columns = java.util.LinkedHashMap() - columns[generator.buildColumnId(prefix + "the_column_name")] = AirbyteProtocolType.STRING - val stream = - StreamConfig( - streamId, - ImportType.APPEND, - emptyList(), - Optional.empty(), - columns, - 0, - 0, - 0 - ) - - val createTable = generator.createTable(stream, "", false) - Assertions.assertThrows(BigQueryException::class.java) { - destinationHandler.execute(createTable) - } + fun toJsonRecords(result: TableResult): List { + return result + .streamAll() + .map { row: FieldValueList -> toJson(result.schema!!, row) } + .toList() } /** - * Something about this test is borked on bigquery. It fails because the raw table doesn't - * exist, but you can go into the UI and see that it does exist. + * FieldValueList stores everything internally as string (I think?) but provides conversions to + * more useful types. This method does that conversion, using the schema to determine which type + * is most appropriate. Then we just dump everything into a jsonnode for interop with + * RecordDiffer. */ - @Disabled - @Throws(Exception::class) - override fun noCrashOnSpecialCharacters(specialChars: String) { - super.noCrashOnSpecialCharacters(specialChars) - } - - /** - * Bigquery doesn't handle frequent INSERT/DELETE statements on a single table very well. So we - * don't have real state handling. Disable this test. - */ - @Disabled - @Test - @Throws(Exception::class) - override fun testStateHandling() { - super.testStateHandling() - } - - @Disabled - override fun testLongIdentifierHandling() { - super.testLongIdentifierHandling() - } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(BigQuerySqlGeneratorIntegrationTest::class.java) - - private var bq: BigQuery? = null - private var projectId: String? = null - private var datasetLocation: String? = null - - @BeforeAll - @Throws(Exception::class) - @JvmStatic - fun setupBigquery() { - val rawConfig = Files.readString(Path.of("secrets/credentials-gcs-staging.json")) - val config = deserialize(rawConfig) - bq = getBigQuery(config) - - projectId = config[BigQueryConsts.CONFIG_PROJECT_ID].asText() - datasetLocation = config[BigQueryConsts.CONFIG_DATASET_LOCATION].asText() - } - - /** - * TableResult contains records in a somewhat nonintuitive format (and it avoids loading - * them all into memory). That's annoying for us since we're working with small test data, - * so just pull everything into a list. - */ - fun toJsonRecords(result: TableResult): List { - return result - .streamAll() - .map { row: FieldValueList -> toJson(result.schema!!, row) } - .toList() - } - - /** - * FieldValueList stores everything internally as string (I think?) but provides conversions - * to more useful types. This method does that conversion, using the schema to determine - * which type is most appropriate. Then we just dump everything into a jsonnode for interop - * with RecordDiffer. - */ - private fun toJson(schema: Schema, row: FieldValueList): JsonNode { - val json = emptyObject() as ObjectNode - for (i in schema.fields.indices) { - val field = schema.fields[i] - val value = row[i] - val typedValue: JsonNode - if (!value.isNull) { - typedValue = - when (field.type.standardType) { - StandardSQLTypeName.BOOL -> jsonNode(value.booleanValue) - StandardSQLTypeName.INT64 -> jsonNode(value.longValue) - StandardSQLTypeName.FLOAT64 -> jsonNode(value.doubleValue) - StandardSQLTypeName.NUMERIC, - StandardSQLTypeName.BIGNUMERIC -> jsonNode(value.numericValue) - StandardSQLTypeName.STRING -> jsonNode(value.stringValue) - StandardSQLTypeName.TIMESTAMP -> - jsonNode(value.timestampInstant.toString()) - StandardSQLTypeName.DATE, - StandardSQLTypeName.DATETIME, - StandardSQLTypeName.TIME -> jsonNode(value.stringValue) - StandardSQLTypeName.JSON -> - jsonNode(deserializeExact(value.stringValue)) - else -> jsonNode(value.stringValue) - } - json.set(field.name, typedValue) - } + private fun toJson(schema: Schema, row: FieldValueList): JsonNode { + val json = emptyObject() as ObjectNode + for (i in schema.fields.indices) { + val field = schema.fields[i] + val value = row[i] + val typedValue: JsonNode + if (!value.isNull) { + typedValue = + when (field.type.standardType) { + StandardSQLTypeName.BOOL -> jsonNode(value.booleanValue) + StandardSQLTypeName.INT64 -> jsonNode(value.longValue) + StandardSQLTypeName.FLOAT64 -> jsonNode(value.doubleValue) + StandardSQLTypeName.NUMERIC, + StandardSQLTypeName.BIGNUMERIC -> jsonNode(value.numericValue) + StandardSQLTypeName.STRING -> jsonNode(value.stringValue) + StandardSQLTypeName.TIMESTAMP -> jsonNode(value.timestampInstant.toString()) + StandardSQLTypeName.DATE, + StandardSQLTypeName.DATETIME, + StandardSQLTypeName.TIME -> jsonNode(value.stringValue) + StandardSQLTypeName.JSON -> jsonNode(deserializeExact(value.stringValue)) + else -> jsonNode(value.stringValue) + } + json.set(field.name, typedValue) } - return json } + return json } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt new file mode 100644 index 0000000000000..ed69c90be8ca0 --- /dev/null +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt @@ -0,0 +1,689 @@ +/* + * Copyright (c) 2023 Airbyte, Inc., all rights reserved. + */ +package io.airbyte.integrations.destination.bigquery.typing_deduping + +import com.google.cloud.bigquery.StandardSQLTypeName +import com.google.common.annotations.VisibleForTesting +import io.airbyte.integrations.base.destination.typing_deduping.* +import io.airbyte.integrations.base.destination.typing_deduping.Array +import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer +import java.time.Instant +import java.util.* +import java.util.stream.Collectors +import java.util.stream.Stream +import org.apache.commons.lang3.StringUtils +import org.slf4j.Logger +import org.slf4j.LoggerFactory + +/** + * This class is just a copy of the old BigQuerySqlGenerator. It's only used for the [buildStreamId] + * function. + */ +class LegacyBigQuerySqlGenerator +/** + * @param projectId + * @param datasetLocation This is technically redundant with [BigQueryDestinationHandler] setting + * the query execution location, but let's be explicit since this is typically a compliance + * requirement. + */ +(private val projectId: String?, private val datasetLocation: String?) : SqlGenerator { + private val CDC_DELETED_AT_COLUMN = buildColumnId("_ab_cdc_deleted_at") + + private val LOGGER: Logger = LoggerFactory.getLogger(LegacyBigQuerySqlGenerator::class.java) + + override fun buildStreamId( + namespace: String, + name: String, + rawNamespaceOverride: String + ): StreamId { + return StreamId( + nameTransformer.getNamespace(namespace), + nameTransformer.convertStreamName(name), + nameTransformer.getNamespace(rawNamespaceOverride), + nameTransformer.convertStreamName(StreamId.concatenateRawTableName(namespace, name)), + namespace, + name + ) + } + + override fun buildColumnId(name: String, suffix: String?): ColumnId { + val nameWithSuffix = name + suffix + return ColumnId( + nameTransformer.getIdentifier(nameWithSuffix), + name, // Bigquery columns are case-insensitive, so do all our validation on the + // lowercased name + nameTransformer.getIdentifier(nameWithSuffix.lowercase(Locale.getDefault())) + ) + } + + private fun extractAndCast( + column: ColumnId, + airbyteType: AirbyteType, + forceSafeCast: Boolean + ): String { + if (airbyteType is Union) { + // This is guaranteed to not be a Union, so we won't recurse infinitely + val chosenType: AirbyteType = airbyteType.chooseType() + return extractAndCast(column, chosenType, forceSafeCast) + } + val columnName = escapeColumnNameForJsonPath(column.originalName) + + if (airbyteType is Struct) { + // We need to validate that the struct is actually a struct. + // Note that struct columns are actually nullable in two ways. For a column `foo`: + // {foo: null} and {} are both valid, and are both written to the final table as a SQL + // NULL (_not_ a + // JSON null). + // JSON_QUERY(JSON'{}', '$."foo"') returns a SQL null. + // JSON_QUERY(JSON'{"foo": null}', '$."foo"') returns a JSON null. + return """ + PARSE_JSON(CASE + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'object' + THEN NULL + ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + END, wide_number_mode=>'round') + """.trimIndent() + } + + if (airbyteType is Array) { + // Much like the Struct case above, arrays need special handling. + return """ + PARSE_JSON(CASE + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'array' + THEN NULL + ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + END, wide_number_mode=>'round') + """.trimIndent() + } + + if (airbyteType is UnsupportedOneOf || airbyteType === AirbyteProtocolType.UNKNOWN) { + // JSON_QUERY returns a SQL null if the field contains a JSON null, so we actually parse + // the + // airbyte_data to json + // and json_query it directly (which preserves nulls correctly). + return """JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$columnName"')""" + } + + if (airbyteType === AirbyteProtocolType.STRING) { + // Special case String to only use json value for type string and parse the json for + // others + // Naive json_value returns NULL for object/array values and json_query adds escaped + // quotes to the + // string. + return """ + (CASE + WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL + OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'string' + THEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') + ELSE + JSON_VALUE(`_airbyte_data`, '${'$'}."$columnName"') + END) + """.trimIndent() + } + + val dialectType = toDialectType(airbyteType) + val baseTyping = """JSON_VALUE(`_airbyte_data`, '$."$columnName"')""" + return if (dialectType == StandardSQLTypeName.STRING) { + // json_value implicitly returns a string, so we don't need to cast it. + baseTyping + } else { + // SAFE_CAST is actually a massive performance hit, so we should skip it if we can. + cast(baseTyping, dialectType.name, forceSafeCast) + } + } + + override fun createTable(stream: StreamConfig, suffix: String, force: Boolean): Sql { + val columnDeclarations = columnsAndTypes(stream) + val clusterConfig = + clusteringColumns(stream) + .stream() + .map { c: String? -> StringUtils.wrap(c, QUOTE) } + .collect(Collectors.joining(", ")) + val forceCreateTable = if (force) "OR REPLACE" else "" + val finalTableId = stream.id.finalTableId(QUOTE, suffix) + return Sql.of( + """ + CREATE $forceCreateTable TABLE `$projectId`.$finalTableId ( + _airbyte_raw_id STRING NOT NULL, + _airbyte_extracted_at TIMESTAMP NOT NULL, + _airbyte_meta JSON NOT NULL, + _airbyte_generation_id INTEGER, + $columnDeclarations + ) + PARTITION BY (DATE_TRUNC(_airbyte_extracted_at, DAY)) + CLUSTER BY $clusterConfig; + """.trimIndent() + ) + } + + private fun columnsAndTypes(stream: StreamConfig): String { + return stream.columns.entries + .stream() + .map { column: Map.Entry -> + java.lang.String.join(" ", column.key.name(QUOTE), toDialectType(column.value).name) + } + .collect(Collectors.joining(",\n")) + } + + override fun prepareTablesForSoftReset(stream: StreamConfig): Sql { + // Bigquery can't run DDL in a transaction, so these are separate transactions. + return Sql + .concat( // If a previous sync failed to delete the soft reset temp table (unclear why + // this happens), + // AND this sync is trying to change the clustering config, then we need to manually + // drop the soft + // reset temp table. + // Even though we're using CREATE OR REPLACE TABLE, bigquery will still complain + // about the + // clustering config being changed. + // So we explicitly drop the soft reset temp table first. + dropTableIfExists(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX), + createTable(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX, true), + clearLoadedAt(stream.id) + ) + } + + private fun dropTableIfExists(stream: StreamConfig, suffix: String): Sql { + val tableId = stream.id.finalTableId(QUOTE, suffix) + return Sql.of("""DROP TABLE IF EXISTS `$projectId`.$tableId;""") + } + + override fun clearLoadedAt(streamId: StreamId): Sql { + val rawTableId = streamId.rawTableId(QUOTE) + return Sql.of( + """UPDATE `$projectId`.$rawTableId SET _airbyte_loaded_at = NULL WHERE 1=1;""" + ) + } + + override fun updateTable( + stream: StreamConfig, + finalSuffix: String, + minRawTimestamp: Optional, + useExpensiveSaferCasting: Boolean + ): Sql { + val handleNewRecords = + if (stream.postImportAction == ImportType.DEDUPE) { + upsertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp) + } else { + insertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp) + } + val commitRawTable = commitRawTable(stream.id, minRawTimestamp) + + return Sql.transactionally(handleNewRecords, commitRawTable) + } + + private fun insertNewRecords( + stream: StreamConfig, + finalSuffix: String, + forceSafeCasting: Boolean, + minRawTimestamp: Optional + ): String { + val columnList: String = + stream.columns.keys + .stream() + .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } + .collect(Collectors.joining("\n")) + val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp) + val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix) + + return """ + INSERT INTO `$projectId`.$finalTableId + ( + $columnList + _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_generation_id + ) + $extractNewRawRecords; + """.trimIndent() + } + + private fun upsertNewRecords( + stream: StreamConfig, + finalSuffix: String, + forceSafeCasting: Boolean, + minRawTimestamp: Optional + ): String { + val pkEquivalent = + stream.primaryKey + .stream() + .map { pk: ColumnId -> + val quotedPk = pk.name(QUOTE) + ("""(target_table.$quotedPk = new_record.$quotedPk OR (target_table.$quotedPk IS NULL AND new_record.$quotedPk IS NULL))""") + } + .collect(Collectors.joining(" AND ")) + + val columnList: String = + stream.columns.keys + .stream() + .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } + .collect(Collectors.joining("\n")) + val newRecordColumnList: String = + stream.columns.keys + .stream() + .map { quotedColumnId: ColumnId -> + "new_record." + quotedColumnId.name(QUOTE) + "," + } + .collect(Collectors.joining("\n")) + val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp) + + val cursorComparison: String + if (stream.cursor.isPresent) { + val cursor = stream.cursor.get().name(QUOTE) + // Build a condition for "new_record is more recent than target_table": + cursorComparison = // First, compare the cursors. + (""" + ( + target_table.$cursor < new_record.$cursor + OR (target_table.$cursor = new_record.$cursor AND target_table._airbyte_extracted_at < new_record._airbyte_extracted_at) + OR (target_table.$cursor IS NULL AND new_record.$cursor IS NULL AND target_table._airbyte_extracted_at < new_record._airbyte_extracted_at) + OR (target_table.$cursor IS NULL AND new_record.$cursor IS NOT NULL) + ) + """.trimIndent()) + } else { + // If there's no cursor, then we just take the most-recently-emitted record + cursorComparison = + "target_table._airbyte_extracted_at < new_record._airbyte_extracted_at" + } + + val cdcDeleteClause: String + val cdcSkipInsertClause: String + if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) { + // Execute CDC deletions if there's already a record + cdcDeleteClause = + "WHEN MATCHED AND new_record._ab_cdc_deleted_at IS NOT NULL AND $cursorComparison THEN DELETE" + // And skip insertion entirely if there's no matching record. + // (This is possible if a single T+D batch contains both an insertion and deletion for + // the same PK) + cdcSkipInsertClause = "AND new_record._ab_cdc_deleted_at IS NULL" + } else { + cdcDeleteClause = "" + cdcSkipInsertClause = "" + } + + val columnAssignments: String = + stream.columns.keys + .stream() + .map { airbyteType: ColumnId -> + val column = airbyteType.name(QUOTE) + "$column = new_record.$column," + } + .collect(Collectors.joining("\n")) + val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix) + + return """ + MERGE `$projectId`.$finalTableId target_table + USING ( + $extractNewRawRecords + ) new_record + ON $pkEquivalent + $cdcDeleteClause + WHEN MATCHED AND $cursorComparison THEN UPDATE SET + $columnAssignments + _airbyte_meta = new_record._airbyte_meta, + _airbyte_raw_id = new_record._airbyte_raw_id, + _airbyte_extracted_at = new_record._airbyte_extracted_at, + _airbyte_generation_id = new_record._airbyte_generation_id + WHEN NOT MATCHED $cdcSkipInsertClause THEN INSERT ( + $columnList + _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_generation_id + ) VALUES ( + $newRecordColumnList + new_record._airbyte_meta, + new_record._airbyte_raw_id, + new_record._airbyte_extracted_at, + new_record._airbyte_generation_id + ); + """.trimIndent() + } + + /** + * A SQL SELECT statement that extracts new records from the raw table, casts their columns, and + * builds their airbyte_meta column. + * + * In dedup mode: Also extracts all raw CDC deletion records (for tombstoning purposes) and + * dedupes the records (since we only need the most-recent record to upsert). + */ + private fun extractNewRawRecords( + stream: StreamConfig, + forceSafeCasting: Boolean, + minRawTimestamp: Optional + ): String { + val columnCasts: String = + stream.columns.entries + .stream() + .map { col: Map.Entry -> + val extractAndCast = extractAndCast(col.key, col.value, forceSafeCasting) + val columnName = col.key.name(QUOTE) + """$extractAndCast as $columnName,""" + } + .collect(Collectors.joining("\n")) + val columnErrors = + if (forceSafeCasting) { + "[" + + stream.columns.entries + .stream() + .map { col: Map.Entry -> + val rawColName = escapeColumnNameForJsonPath(col.key.originalName) + val jsonExtract = extractAndCast(col.key, col.value, true) + // Explicitly parse json here. This is safe because + // we're not using the actual value anywhere, + // and necessary because json_query + """ + CASE + WHEN (JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$rawColName"') IS NOT NULL) + AND (JSON_TYPE(JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$rawColName"')) != 'null') + AND ($jsonExtract IS NULL) + THEN JSON '{"field":"$rawColName","change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}' + ELSE NULL + END + """.trimIndent() + } + .collect(Collectors.joining(",\n")) + + "]" + } else { + // We're not safe casting, so any error should throw an exception and trigger the + // safe cast logic + "[]" + } + + val columnList: String = + stream.columns.keys + .stream() + .map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," } + .collect(Collectors.joining("\n")) + val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) + + val rawTableId = stream.id.rawTableId(QUOTE) + if (stream.postImportAction == ImportType.DEDUPE) { + // When deduping, we need to dedup the raw records. Note the row_number() invocation in + // the SQL + // statement. Do the same extract+cast CTE + airbyte_meta construction as in non-dedup + // mode, but + // then add a row_number column so that we only take the most-recent raw record for each + // PK. + + // We also explicitly include old CDC deletion records, which act as tombstones to + // correctly delete + // out-of-order records. + + var cdcConditionalOrIncludeStatement = "" + if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) { + cdcConditionalOrIncludeStatement = + """ + OR ( + _airbyte_loaded_at IS NOT NULL + AND JSON_VALUE(`_airbyte_data`, '${'$'}._ab_cdc_deleted_at') IS NOT NULL + ) + """.trimIndent() + } + + val pkList = + stream.primaryKey + .stream() + .map { columnId: ColumnId -> columnId.name(QUOTE) } + .collect(Collectors.joining(",")) + val cursorOrderClause = + stream.cursor + .map { cursorId: ColumnId -> cursorId.name(QUOTE) + " DESC NULLS LAST," } + .orElse("") + + return """ + WITH intermediate_data AS ( + SELECT + $columnCasts + $columnErrors AS column_errors, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_meta, + _airbyte_generation_id + FROM `$projectId`.$rawTableId + WHERE ( + _airbyte_loaded_at IS NULL + $cdcConditionalOrIncludeStatement + ) $extractedAtCondition + ), new_records AS ( + SELECT + $columnList + to_json(json_set( + coalesce(parse_json(_airbyte_meta), JSON'{}'), + '${'$'}.changes', + json_array_append( + coalesce(json_query(parse_json(_airbyte_meta), '${'$'}.changes'), JSON'[]'), + '${'$'}', + COALESCE((SELECT ARRAY_AGG(unnested_column_errors IGNORE NULLS) FROM UNNEST(column_errors) unnested_column_errors), []) + ) + )) as _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_generation_id + FROM intermediate_data + ), numbered_rows AS ( + SELECT *, row_number() OVER ( + PARTITION BY $pkList ORDER BY $cursorOrderClause `_airbyte_extracted_at` DESC + ) AS row_number + FROM new_records + ) + SELECT $columnList _airbyte_meta, _airbyte_raw_id, _airbyte_extracted_at, _airbyte_generation_id + FROM numbered_rows + WHERE row_number = 1 + """.trimIndent() + } else { + // When not deduplicating, we just need to handle type casting. + // Extract+cast the not-yet-loaded records in a CTE, then select that CTE and build + // airbyte_meta. + + return """ + WITH intermediate_data AS ( + SELECT + $columnCasts + $columnErrors AS column_errors, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_meta, + _airbyte_generation_id + FROM `$projectId`.$rawTableId + WHERE + _airbyte_loaded_at IS NULL + $extractedAtCondition + ) + SELECT + $columnList + to_json(json_set( + coalesce(parse_json(_airbyte_meta), JSON'{}'), + '${'$'}.changes', + json_array_append( + coalesce(json_query(parse_json(_airbyte_meta), '${'$'}.changes'), JSON'[]'), + '${'$'}', + COALESCE((SELECT ARRAY_AGG(unnested_column_errors IGNORE NULLS) FROM UNNEST(column_errors) unnested_column_errors), []) + ) + )) as _airbyte_meta, + _airbyte_raw_id, + _airbyte_extracted_at, + _airbyte_generation_id + FROM intermediate_data + """.trimIndent() + } + } + + @VisibleForTesting + fun commitRawTable(id: StreamId, minRawTimestamp: Optional): String { + val rawTableId = id.rawTableId(QUOTE) + val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) + return """ + UPDATE `$projectId`.$rawTableId + SET `_airbyte_loaded_at` = CURRENT_TIMESTAMP() + WHERE `_airbyte_loaded_at` IS NULL + $extractedAtCondition + ; + """.trimIndent() + } + + override fun overwriteFinalTable(stream: StreamId, finalSuffix: String): Sql { + val finalTableId = stream.finalTableId(QUOTE) + val tempFinalTableId = stream.finalTableId(QUOTE, finalSuffix) + val realFinalTableName = stream.finalName(QUOTE) + return Sql.separately( + "DROP TABLE IF EXISTS `$projectId`.$finalTableId;", + "ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO $realFinalTableName;" + ) + } + + private fun wrapAndQuote(namespace: String, tableName: String): String { + return Stream.of(namespace, tableName) + .map { part: String? -> StringUtils.wrap(part, QUOTE) } + .collect(Collectors.joining(".")) + } + + override fun createSchema(schema: String): Sql { + val projectId = StringUtils.wrap(projectId, QUOTE) + val quotedSchema = StringUtils.wrap(schema, QUOTE) + return Sql.of( + """CREATE SCHEMA IF NOT EXISTS $projectId.$quotedSchema OPTIONS(location="$datasetLocation");""" + ) + } + + override fun migrateFromV1toV2(streamId: StreamId, namespace: String, tableName: String): Sql { + val v2RawTable = streamId.rawTableId(QUOTE) + val v1RawTable = wrapAndQuote(namespace, tableName) + return Sql.of( + """ + CREATE OR REPLACE TABLE `$projectId`.$v2RawTable ( + _airbyte_raw_id STRING, + _airbyte_data STRING, + _airbyte_extracted_at TIMESTAMP, + _airbyte_loaded_at TIMESTAMP, + _airbyte_meta STRING, + _airbyte_generation_id INTEGER + ) + PARTITION BY DATE(_airbyte_extracted_at) + CLUSTER BY _airbyte_extracted_at + AS ( + SELECT + _airbyte_ab_id AS _airbyte_raw_id, + _airbyte_data AS _airbyte_data, + _airbyte_emitted_at AS _airbyte_extracted_at, + CAST(NULL AS TIMESTAMP) AS _airbyte_loaded_at, + '{"sync_id": 0, "changes": []}' AS _airbyte_meta, + 0 as _airbyte_generation_id + FROM `$projectId`.$v1RawTable + ); + """.trimIndent() + ) + } + + /** + * Does two things: escape single quotes (for use inside sql string literals),and escape double + * quotes (for use inside JSON paths). For example, if a column name is foo'bar"baz, then we + * want to end up with something like `SELECT JSON_QUERY(..., '$."foo\'bar\\"baz"')`. Note the + * single-backslash for single-quotes (needed for SQL) and the double-backslash for + * double-quotes (needed for JSON path). + */ + private fun escapeColumnNameForJsonPath(stringContents: String): String { + // This is not a place of honor. + return stringContents // Consider the JSON blob {"foo\\bar": 42}. + // This is an object with key foo\bar. + // The JSONPath for this is $."foo\\bar" (i.e. 2 backslashes to represent the single + // backslash in the key). + // When we represent that path as a SQL string, the backslashes are doubled (to 4): + // '$."foo\\\\bar"' + // And we're writing that in a Java string, so we have to type out 8 backslashes: + // "'$.\"foo\\\\\\\\bar\"'" + .replace("\\", "\\\\\\\\") // Similar situation here: + // a literal " needs to be \" in a JSONPath: $."foo\"bar" + // which is \\" in a SQL string: '$."foo\\"bar"' + // The backslashes become \\\\ in java, and the quote becomes \": "'$.\"foo\\\\\"bar\"'" + .replace( + "\"", + "\\\\\"" + ) // Here we're escaping a SQL string, so we only need a single backslash (which is 2, + // because Java). + .replace("'", "\\'") + } + + companion object { + const val QUOTE: String = "`" + private val nameTransformer = BigQuerySQLNameTransformer() + + @JvmStatic + fun toDialectType(type: AirbyteType): StandardSQLTypeName { + // switch pattern-matching is still in preview at language level 17 :( + if (type is AirbyteProtocolType) { + return toDialectType(type) + } else if (type is Struct) { + return StandardSQLTypeName.JSON + } else if (type is Array) { + return StandardSQLTypeName.JSON + } else if (type is UnsupportedOneOf) { + return StandardSQLTypeName.JSON + } else if (type is Union) { + val typeWithPrecedence: AirbyteType = type.chooseType() + val dialectType: StandardSQLTypeName + if ((typeWithPrecedence is Struct) || (typeWithPrecedence is Array)) { + dialectType = StandardSQLTypeName.JSON + } else { + dialectType = toDialectType(typeWithPrecedence as AirbyteProtocolType) + } + return dialectType + } + + // Literally impossible; AirbyteType is a sealed interface. + throw IllegalArgumentException("Unsupported AirbyteType: $type") + } + + // TODO maybe make this a BiMap and elevate this method and its inverse + // (toDestinationSQLType?) to + // the SQLGenerator? + fun toDialectType(airbyteProtocolType: AirbyteProtocolType): StandardSQLTypeName { + return when (airbyteProtocolType) { + AirbyteProtocolType.STRING, + AirbyteProtocolType.TIME_WITH_TIMEZONE -> StandardSQLTypeName.STRING + AirbyteProtocolType.NUMBER -> StandardSQLTypeName.NUMERIC + AirbyteProtocolType.INTEGER -> StandardSQLTypeName.INT64 + AirbyteProtocolType.BOOLEAN -> StandardSQLTypeName.BOOL + AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE -> StandardSQLTypeName.TIMESTAMP + AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE -> StandardSQLTypeName.DATETIME + AirbyteProtocolType.TIME_WITHOUT_TIMEZONE -> StandardSQLTypeName.TIME + AirbyteProtocolType.DATE -> StandardSQLTypeName.DATE + AirbyteProtocolType.UNKNOWN -> StandardSQLTypeName.JSON + } + } + + fun clusteringColumns(stream: StreamConfig): List { + val clusterColumns: MutableList = ArrayList() + if (stream.postImportAction == ImportType.DEDUPE) { + // We're doing de-duping, therefore we have a primary key. + // Cluster on the first 3 PK columns since BigQuery only allows up to 4 clustering + // columns, + // and we're always clustering on _airbyte_extracted_at + stream.primaryKey.stream().limit(3).forEach { columnId: ColumnId -> + clusterColumns.add(columnId.name) + } + } + clusterColumns.add("_airbyte_extracted_at") + return clusterColumns + } + + private fun buildExtractedAtCondition(minRawTimestamp: Optional): String { + return minRawTimestamp + .map { ts: Instant -> " AND _airbyte_extracted_at > '$ts'" } + .orElse("") + } + + private fun cast(content: String, asType: String, useSafeCast: Boolean): String { + val open = if (useSafeCast) "SAFE_CAST(" else "CAST(" + return wrap(open, "$content as $asType", ")") + } + + private fun wrap(open: String, content: String, close: String): String { + return open + content + close + } + } +} From f90b2b318faa6b63a0e735902cfe49f3f9c13ec0 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Fri, 18 Apr 2025 16:29:39 -0700 Subject: [PATCH 24/55] kill streamid here --- .../bigquery/typing_deduping/BigqueryNameGenerators.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt index 6a8d6c8f2b2ed..ac898ec8a4620 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt @@ -9,7 +9,7 @@ import io.airbyte.cdk.load.orchestration.ColumnNameGenerator import io.airbyte.cdk.load.orchestration.FinalTableNameGenerator import io.airbyte.cdk.load.orchestration.RawTableNameGenerator import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.integrations.base.destination.typing_deduping.StreamId +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingUtil import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer import java.util.Locale @@ -21,7 +21,7 @@ class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTabl TableName( nameTransformer.getNamespace(config.rawTableDataset), nameTransformer.convertStreamName( - StreamId.concatenateRawTableName( + TypingDedupingUtil.concatenateRawTableName( streamDescriptor.namespace ?: config.datasetId, streamDescriptor.name, ) From 9e1a737cc647707cbe40a3cbd06fb9c77b893438 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Fri, 18 Apr 2025 16:30:52 -0700 Subject: [PATCH 25/55] kill files --- .../bigquery/BigQueryConsumerFactory.kt | 67 ------------- .../migrators/BigQueryDestinationState.kt | 18 ---- ...ueryAirbyteMetaAndGenerationIdMigration.kt | 97 ------------------- .../typing_deduping/BigQueryV1V2Migrator.kt | 60 ------------ 4 files changed, 242 deletions(-) delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryConsumerFactory.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDestinationState.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigqueryAirbyteMetaAndGenerationIdMigration.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryV1V2Migrator.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryConsumerFactory.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryConsumerFactory.kt deleted file mode 100644 index 1e2acc7b76894..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryConsumerFactory.kt +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery - -import io.airbyte.cdk.integrations.destination.async.AsyncStreamConsumer -import io.airbyte.cdk.integrations.destination.async.buffers.BufferManager -import io.airbyte.cdk.integrations.destination.operation.SyncOperation -import io.airbyte.integrations.base.destination.operation.DefaultFlush -import io.airbyte.protocol.models.v0.AirbyteMessage -import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog -import java.util.* -import java.util.function.Consumer - -object BigQueryConsumerFactory { - - fun createStagingConsumer( - outputRecordCollector: Consumer, - syncOperation: SyncOperation, - catalog: ConfiguredAirbyteCatalog, - defaultNamespace: String - ): AsyncStreamConsumer { - // values here are resurrected from some old code. - // TODO: Find why max memory ratio is 0.4 capped - return AsyncStreamConsumer( - outputRecordCollector = outputRecordCollector, - onStart = {}, - onClose = { _, streamSyncSummaries -> - syncOperation.finalizeStreams(streamSyncSummaries) - }, - onFlush = DefaultFlush(200 * 1024 * 1024, syncOperation), - catalog = catalog, - bufferManager = - BufferManager( - defaultNamespace, - (Runtime.getRuntime().maxMemory() * 0.4).toLong(), - ), - ) - } - - fun createDirectUploadConsumer( - outputRecordCollector: Consumer, - syncOperation: SyncOperation, - catalog: ConfiguredAirbyteCatalog, - defaultNamespace: String - ): AsyncStreamConsumer { - - // TODO: Why is Standard consumer operating at memory ratio of 0.5 - // and Max 2 threads and some weird 20% max memory as the default flush size. - return AsyncStreamConsumer( - outputRecordCollector = outputRecordCollector, - onStart = {}, - onClose = { _, streamSyncSummaries -> - syncOperation.finalizeStreams(streamSyncSummaries) - }, - onFlush = - DefaultFlush((Runtime.getRuntime().maxMemory() * 0.2).toLong(), syncOperation), - catalog = catalog, - bufferManager = - BufferManager( - defaultNamespace, - (Runtime.getRuntime().maxMemory() * 0.5).toLong(), - ), - ) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDestinationState.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDestinationState.kt deleted file mode 100644 index 5517707f35421..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigQueryDestinationState.kt +++ /dev/null @@ -1,18 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.migrators - -import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState - -data class BigQueryDestinationState(private val needsSoftReset: Boolean) : MinimumDestinationState { - override fun needsSoftReset(): Boolean { - return needsSoftReset - } - - @Suppress("UNCHECKED_CAST") - override fun withSoftReset(needsSoftReset: Boolean): T { - return copy(needsSoftReset = needsSoftReset) as T - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigqueryAirbyteMetaAndGenerationIdMigration.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigqueryAirbyteMetaAndGenerationIdMigration.kt deleted file mode 100644 index 8c772bdb1e6a3..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/migrators/BigqueryAirbyteMetaAndGenerationIdMigration.kt +++ /dev/null @@ -1,97 +0,0 @@ -/* - * Copyright (c) 2024 Airbyte, Inc., all rights reserved. - */ - -package io.airbyte.integrations.destination.bigquery.migrators - -import com.google.cloud.bigquery.BigQuery -import com.google.cloud.bigquery.Field -import com.google.cloud.bigquery.Schema -import com.google.cloud.bigquery.StandardSQLTypeName -import com.google.cloud.bigquery.StandardTableDefinition -import com.google.cloud.bigquery.TableId -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 - -class BigqueryAirbyteMetaAndGenerationIdMigration(private val bigquery: BigQuery) : - Migration { - private val logger = KotlinLogging.logger {} - - override fun migrateIfNecessary( - destinationHandler: DestinationHandler, - stream: StreamConfig, - state: DestinationInitialStatus - ): Migration.MigrationResult { - if (!state.initialRawTableStatus.rawTableExists) { - // The raw table doesn't exist. No migration necessary. Update the state. - logger.info { - "Skipping airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because the raw table doesn't exist" - } - return Migration.MigrationResult(state.destinationState, false) - } - - val rawTable = bigquery.getTable(TableId.of(stream.id.rawNamespace, stream.id.rawName)) - // if the schema is null, then we have bigger problems - val rawFields = rawTable.getDefinition().schema!!.fields - val hasMeta = rawFields.any { it.name == JavaBaseConstants.COLUMN_NAME_AB_META } - if (hasMeta) { - // We've already executed the migration. Do nothing here. - logger.info { - "Skipping airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because the table already has the columns" - } - return Migration.MigrationResult(state.destinationState, false) - } - - logger.info { - "Executing airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName}" - } - - // Define the new columns we're adding. - // Add meta to the raw table - val rawMetaField = - Field.of(JavaBaseConstants.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING) - // And add generation ID to raw+final tables. - val generationIdField = - Field.of(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64) - - // The way bigquery does an "alter table" is by just setting the entire table definition - // (unless you want to write actual SQL, of course). - // This adds the column as NULLABLE. - val newRawTable = - rawTable - .toBuilder() - .setDefinition( - StandardTableDefinition.of( - Schema.of(rawFields + rawMetaField + generationIdField) - ) - ) - .build() - newRawTable.update() - - if (state.isFinalTablePresent) { - val finalTable = - bigquery.getTable(TableId.of(stream.id.finalNamespace, stream.id.finalName)) - val finalFields = finalTable.getDefinition().schema!!.fields - val airbyteMetaIndex = - finalFields.indexOfFirst { it.name == JavaBaseConstants.COLUMN_NAME_AB_META } - // Insert generation_id immediately after airbyte_meta - val newFinalFields = - finalFields.subList(0, airbyteMetaIndex + 1) + - generationIdField + - finalFields.subList(airbyteMetaIndex + 1, finalFields.size) - val newFinalTable = - finalTable - .toBuilder() - .setDefinition(StandardTableDefinition.of(Schema.of(newFinalFields))) - .build() - newFinalTable.update() - } - - // We need to refetch the initial state, because we modified the final table schema. - return Migration.MigrationResult(state.destinationState, true) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryV1V2Migrator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryV1V2Migrator.kt deleted file mode 100644 index a4f6548c25750..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryV1V2Migrator.kt +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ -package io.airbyte.integrations.destination.bigquery.typing_deduping - -import com.google.cloud.bigquery.* -import edu.umd.cs.findbugs.annotations.SuppressFBWarnings -import io.airbyte.integrations.base.destination.typing_deduping.BaseDestinationV1V2Migrator -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase -import io.airbyte.integrations.base.destination.typing_deduping.NamespacedTableName -import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig -import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer -import java.util.* -import java.util.stream.Collectors - -class BigQueryV1V2Migrator( - private val bq: BigQuery, - private val nameTransformer: BigQuerySQLNameTransformer -) : BaseDestinationV1V2Migrator() { - @SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE") - override fun doesAirbyteInternalNamespaceExist(streamConfig: StreamConfig?): Boolean { - val dataset = bq.getDataset(streamConfig!!.id.rawNamespace) - return dataset != null && dataset.exists() - } - - override fun getTableIfExists( - namespace: String?, - tableName: String? - ): Optional { - val table = bq.getTable(TableId.of(namespace, tableName)) - return if (table != null && table.exists()) Optional.of(table.getDefinition()) - else Optional.empty() - } - - override fun schemaMatchesExpectation( - existingTable: TableDefinition, - columns: Collection - ): Boolean { - val existingSchemaColumns = - Optional.ofNullable(existingTable.schema) - .map { schema: Schema -> - schema.fields - .stream() - .map { obj: Field -> obj.name } - .collect(Collectors.toSet()) - } - .orElse(emptySet()) - - return !existingSchemaColumns.isEmpty() && - containsAllIgnoreCase(columns, existingSchemaColumns) - } - - @Suppress("deprecation") - override fun convertToV1RawName(streamConfig: StreamConfig): NamespacedTableName { - return NamespacedTableName( - nameTransformer.getNamespace(streamConfig.id.originalNamespace), - nameTransformer.getRawTableName(streamConfig.id.originalName) - ) - } -} From b1c89f027a7492e19a3cfc252b4ebc306dbe1f74 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Fri, 18 Apr 2025 16:34:32 -0700 Subject: [PATCH 26/55] one usage --- .../BigqueryDestinationInitialStatusGatherer.kt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index 5010fa23f424f..aca7c04efa5ba 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -26,7 +26,6 @@ import io.airbyte.cdk.load.orchestration.TableNames import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.FinalTableInitialStatus import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.RawTableInitialStatus import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus -import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation import io.airbyte.integrations.base.destination.typing_deduping.AlterTableReport import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsIgnoreCase @@ -141,7 +140,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : val tempRawTableState = getInitialRawTableState( tableNames.rawTableName!!, - AbstractStreamOperation.TMP_TABLE_SUFFIX + TableNames.TMP_TABLE_SUFFIX, ) TypingDedupingDestinationInitialStatus( finalTableStatus, From f5df055810050a016a09bd86ac3cbc2f06545ad5 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Fri, 18 Apr 2025 16:37:19 -0700 Subject: [PATCH 27/55] whoops --- .../TypingDedupingUtil.kt | 31 +++++++++++++++++++ 1 file changed, 31 insertions(+) create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt new file mode 100644 index 0000000000000..a772295f76381 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt @@ -0,0 +1,31 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +import kotlin.math.max + +object TypingDedupingUtil { + // copied wholesale from old CDK's StreamId + fun concatenateRawTableName(namespace: String, name: String): String { + val plainConcat = namespace + name + // Pretend we always have at least one underscore, so that we never generate + // `_raw_stream_` + var longestUnderscoreRun = 1 + var i = 0 + while (i < plainConcat.length) { + // If we've found an underscore, count the number of consecutive underscores + var underscoreRun = 0 + while (i < plainConcat.length && plainConcat[i] == '_') { + underscoreRun++ + i++ + } + longestUnderscoreRun = + max(longestUnderscoreRun.toDouble(), underscoreRun.toDouble()).toInt() + i++ + } + + return namespace + "_raw" + "_".repeat(longestUnderscoreRun + 1) + "stream_" + name + } +} From b54cebdd7409edfc87436b02948b3bf86bb64fd1 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:19:03 -0700 Subject: [PATCH 28/55] copy AlterTableReport + CollectionUtils --- .../io/airbyte/cdk/util/CollectionUtils.kt | 62 +++++++++++++++++++ .../AlterTableReport.kt | 19 ++++++ ...igqueryDestinationInitialStatusGatherer.kt | 26 ++------ 3 files changed, 85 insertions(+), 22 deletions(-) create mode 100644 airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/CollectionUtils.kt create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt diff --git a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/CollectionUtils.kt b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/CollectionUtils.kt new file mode 100644 index 0000000000000..cb41277488b14 --- /dev/null +++ b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/CollectionUtils.kt @@ -0,0 +1,62 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.util + +import java.util.Optional + +object CollectionUtils { + /** + * Pass in a collection and search term to determine whether any of the values match ignoring + * case + * + * @param collection the collection of values + * @param search the value to look for + * @return whether the value matches anything in the collection + */ + @JvmStatic + fun containsIgnoreCase(collection: Collection, search: String): Boolean { + return matchingKey(collection, search).isPresent + } + + /** + * Convenience method for when you need to check an entire collection for membership in another + * collection. + * + * @param searchCollection the collection you want to check membership in + * @param searchTerms the keys you're looking for + * @return whether all searchTerms are in the searchCollection + */ + @JvmStatic + fun containsAllIgnoreCase( + searchCollection: Collection, + searchTerms: Collection + ): Boolean { + require(!searchTerms.isEmpty()) { + // There isn't a good behavior for an empty collection. Without this check, an empty + // collection + // would always return + // true, but it feels misleading to say that the searchCollection does "contain all" + // when + // searchTerms is empty + "Search Terms collection may not be empty" + } + return searchTerms.all { term: String -> containsIgnoreCase(searchCollection, term) } + } + + /** + * From a collection of strings, return an entry which matches the search term ignoring case + * + * @param collection the collection to search + * @param search the key you're looking for + * @return an Optional value which might contain the key that matches the search + */ + @JvmStatic + fun matchingKey(collection: Collection, search: String): Optional { + if (collection.contains(search)) { + return Optional.of(search) + } + return Optional.ofNullable(collection.firstOrNull { it.equals(search, ignoreCase = true) }) + } +} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt new file mode 100644 index 0000000000000..70dcba5fbfdae --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt @@ -0,0 +1,19 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.legacy_typing_deduping + +data class AlterTableReport( + val columnsToAdd: Set, + val columnsToRemove: Set, + val columnsToChangeType: Set, +) { + /** + * A no-op for an AlterTableReport is when the existing table matches the expected schema + * + * @return whether the schema matches + */ + val isNoOp = + columnsToAdd.isEmpty() && columnsToRemove.isEmpty() && columnsToChangeType.isEmpty() +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index aca7c04efa5ba..8ce0ea5eae605 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -23,13 +23,13 @@ import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.AlterTableReport import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.FinalTableInitialStatus import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.RawTableInitialStatus import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus -import io.airbyte.integrations.base.destination.typing_deduping.AlterTableReport -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsIgnoreCase -import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.matchingKey +import io.airbyte.cdk.util.CollectionUtils.containsAllIgnoreCase +import io.airbyte.cdk.util.CollectionUtils.containsIgnoreCase +import io.airbyte.cdk.util.CollectionUtils.matchingKey import io.github.oshai.kotlinlogging.KotlinLogging import java.math.BigInteger import java.util.stream.Collectors @@ -231,13 +231,10 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : ) .collect(Collectors.toSet()) - val isDestinationV2Format = schemaContainAllFinalTableV2AirbyteColumns(existingSchema.keys) - return AlterTableReport( columnsToAdd, columnsToRemove, columnsToChangeType, - isDestinationV2Format ) } @@ -263,21 +260,6 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : TimePartitioning.Type.DAY == existingTable.timePartitioning!!.type } - /** - * Checks the schema to determine whether the table contains all expected final table airbyte - * columns - * - * @param columnNames the column names of the schema to check - * @return whether all the [JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS] are present - */ - @VisibleForTesting - fun schemaContainAllFinalTableV2AirbyteColumns(columnNames: Collection?): Boolean { - return JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.stream().allMatch { column: String? - -> - containsIgnoreCase(columnNames!!, column!!) - } - } - private fun getPks( stream: DestinationStream, columnNameMapping: ColumnNameMapping From acbedf0571c1c87fe856bfd1c40a2a08782d057b Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:28:10 -0700 Subject: [PATCH 29/55] kill name transformer referencens --- ...stractBigQueryDestinationAcceptanceTest.kt | 221 ------------------ .../bigquery/BigQueryDestinationTest.kt | 3 - .../BigQueryGcsDestinationAcceptanceTest.kt | 69 ------ ...gQueryStandardDestinationAcceptanceTest.kt | 50 ---- .../LegacyBigQuerySqlGenerator.kt | 7 +- .../bigquery/BigQueryDestination.kt | 10 +- .../bigquery/BigQuerySQLNameTransformer.kt | 21 +- .../typing_deduping/BigqueryNameGenerators.kt | 4 +- .../BigQuerySQLNameTransformerTest.kt | 21 -- 9 files changed, 26 insertions(+), 380 deletions(-) delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.kt delete mode 100644 airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryStandardDestinationAcceptanceTest.kt diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt deleted file mode 100644 index 4aaa49582d4f5..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/AbstractBigQueryDestinationAcceptanceTest.kt +++ /dev/null @@ -1,221 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ -package io.airbyte.integrations.destination.bigquery - -import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.ObjectNode -import com.google.cloud.bigquery.* -import com.google.common.collect.Streams -import io.airbyte.cdk.db.bigquery.BigQueryResultSet -import io.airbyte.cdk.db.bigquery.BigQuerySourceOperations -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.cdk.integrations.destination.NamingConventionTransformer -import io.airbyte.cdk.integrations.destination.StandardNameTransformer -import io.airbyte.cdk.integrations.standardtest.destination.DestinationAcceptanceTest -import io.airbyte.cdk.integrations.standardtest.destination.TestingNamespaces.isOlderThan2Days -import io.airbyte.cdk.integrations.standardtest.destination.comparator.TestDataComparator -import io.airbyte.commons.json.Jsons.deserialize -import io.airbyte.commons.string.Strings.addRandomSuffix -import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery -import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId -import io.airbyte.integrations.destination.bigquery.typing_deduping.LegacyBigQuerySqlGenerator -import java.io.IOException -import java.nio.file.Path -import java.util.* -import java.util.function.Function -import java.util.stream.Collectors -import org.junit.jupiter.api.Assertions -import org.junit.jupiter.api.Disabled -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -@Disabled -abstract class AbstractBigQueryDestinationAcceptanceTest : DestinationAcceptanceTest() { - protected var secretsFile: Path? = null - protected var bigquery: BigQuery? = null - protected var dataset: Dataset? = null - - protected var _config: JsonNode? = null - protected val namingResolver: StandardNameTransformer = StandardNameTransformer() - - override val imageName: String - get() = "airbyte/destination-bigquery:dev" - - override fun getConfig(): JsonNode { - return _config!! - } - - override fun getFailCheckConfig(): JsonNode? { - (_config as ObjectNode?)!!.put(CONFIG_PROJECT_ID, "fake") - return _config - } - - override fun implementsNamespaces(): Boolean { - return true - } - - override fun supportNamespaceTest(): Boolean { - return true - } - - override fun getTestDataComparator(): TestDataComparator { - return BigQueryTestDataComparator() - } - - override fun supportBasicDataTypeTest(): Boolean { - return true - } - - override fun supportArrayDataTypeTest(): Boolean { - return true - } - - override fun supportObjectDataTypeTest(): Boolean { - return true - } - - override fun supportIncrementalSchemaChanges(): Boolean { - return true - } - - override fun getNameTransformer(): Optional { - return Optional.of(NAME_TRANSFORMER) - } - - override fun assertNamespaceNormalization( - testCaseId: String?, - expectedNormalizedNamespace: String?, - actualNormalizedNamespace: String? - ) { - val message = - String.format( - "Test case %s failed; if this is expected, please override assertNamespaceNormalization", - testCaseId - ) - if (testCaseId == "S3A-1") { - /* - * See NamespaceTestCaseProvider for how this suffix is generated.

expectedNormalizedNamespace - * will look something like this: `_99namespace_test_20230824_bicrt`. We want to grab the part after - * `_99namespace`. - */ - val underscoreIndex = expectedNormalizedNamespace!!.indexOf("_", 1) - val randomSuffix = expectedNormalizedNamespace.substring(underscoreIndex) - /* - * bigquery allows originalNamespace starting with a number, and prepending underscore will hide the - * dataset, so we don't do it as we do for other destinations - */ - Assertions.assertEquals("99namespace$randomSuffix", actualNormalizedNamespace, message) - } else { - Assertions.assertEquals(expectedNormalizedNamespace, actualNormalizedNamespace, message) - } - } - - override fun getDefaultSchema(config: JsonNode): String? { - return getDatasetId(config) - } - - @Throws(Exception::class) - override fun retrieveRecords( - env: TestDestinationEnv?, - streamName: String, - namespace: String, - streamSchema: JsonNode - ): List { - val streamId = - LegacyBigQuerySqlGenerator(null, null) - .buildStreamId( - namespace, - streamName, - JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE - ) - return retrieveRecordsFromTable(streamId.rawName, streamId.rawNamespace) - .stream() - .map( - Function { node: JsonNode -> - node.get(JavaBaseConstants.COLUMN_NAME_DATA).asText() - } - ) - .map { jsonString: String? -> deserialize(jsonString) } - .collect(Collectors.toList()) - } - - @Throws(InterruptedException::class) - protected fun retrieveRecordsFromTable(tableName: String?, schema: String?): List { - TimeZone.setDefault(TimeZone.getTimeZone("UTC")) - - val queryConfig = - QueryJobConfiguration.newBuilder( - String.format( - "SELECT * FROM `%s`.`%s` order by %s asc;", - schema, - tableName, - JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT - ) - ) - .setUseLegacySql(false) - .setConnectionProperties( - listOf(ConnectionProperty.of("time_zone", "UTC")) - ) - .build() - - val queryResults = executeQuery(bigquery!!, queryConfig).getLeft().getQueryResults() - val fields = queryResults.schema!!.fields - val sourceOperations = BigQuerySourceOperations() - - return Streams.stream(queryResults.iterateAll()) - .map { fieldValues: FieldValueList -> - sourceOperations.rowToJson(BigQueryResultSet(fieldValues, fields)) - } - .collect(Collectors.toList()) - } - - @Throws(IOException::class) - protected fun setUpBigQuery() { - // secrets file should be set by the inhereting class - Assertions.assertNotNull(secretsFile) - val datasetId = addRandomSuffix("airbyte_tests", "_", 8) - val stagingPathSuffix = addRandomSuffix("test_path", "_", 8) - val config = - BigQueryDestinationTestUtils.createConfig(secretsFile, datasetId, stagingPathSuffix) - - this._config = config - bigquery = BigQueryDestinationTestUtils.initBigQuery(config) - dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId) - } - - protected fun removeOldNamespaces() { - var datasetsDeletedCount = 0 - // todo (cgardens) - hardcoding to testing project to de-risk this running somewhere - // unexpected. - for (dataset1 in - bigquery!! - .listDatasets("dataline-integration-testing", BigQuery.DatasetListOption.all()) - .iterateAll()) { - if (isOlderThan2Days(dataset1.datasetId.dataset)) { - try { - bigquery!!.delete( - dataset1.datasetId, - BigQuery.DatasetDeleteOption.deleteContents() - ) - datasetsDeletedCount++ - } catch (e: BigQueryException) { - LOGGER.error("Failed to delete old dataset: {}", dataset1.datasetId.dataset, e) - } - } - } - LOGGER.info("Deleted {} old datasets.", datasetsDeletedCount) - } - - protected fun tearDownBigQuery() { - BigQueryDestinationTestUtils.tearDownBigQuery(bigquery, dataset, LOGGER) - } - - companion object { - private val NAME_TRANSFORMER: NamingConventionTransformer = BigQuerySQLNameTransformer() - private val LOGGER: Logger = - LoggerFactory.getLogger(AbstractBigQueryDestinationAcceptanceTest::class.java) - - protected const val CONFIG_PROJECT_ID: String = "project_id" - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt index 5695ac7942f17..c526df44ec129 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt @@ -10,7 +10,6 @@ import com.google.common.collect.ImmutableMap import com.google.common.collect.Lists import io.airbyte.cdk.integrations.base.DestinationConfig import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.cdk.integrations.destination.NamingConventionTransformer import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig import io.airbyte.commons.json.Jsons.deserialize import io.airbyte.commons.json.Jsons.jsonNode @@ -427,8 +426,6 @@ internal class BigQueryDestinationTest { ) ) - private val NAMING_RESOLVER: NamingConventionTransformer = BigQuerySQLNameTransformer() - protected var projectId: String? = null protected var datasetId: String? = null protected var config: JsonNode = mock() diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.kt deleted file mode 100644 index b9ede817df20e..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryGcsDestinationAcceptanceTest.kt +++ /dev/null @@ -1,69 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ -package io.airbyte.integrations.destination.bigquery - -import com.amazonaws.services.s3.AmazonS3 -import io.airbyte.cdk.integrations.base.DestinationConfig -import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig -import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getGcsJsonNodeConfig -import java.nio.file.Path -import org.junit.jupiter.api.Disabled -import org.junit.jupiter.api.TestInstance -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -@Disabled -@TestInstance(TestInstance.Lifecycle.PER_CLASS) -class BigQueryGcsDestinationAcceptanceTest : AbstractBigQueryDestinationAcceptanceTest() { - private var s3Client: AmazonS3? = null - - /** - * Sets up secretsFile path as well as BigQuery and GCS instances for verification and cleanup - * This function will be called before EACH test. - * - * @param testEnv - * - information about the test environment. - * @param TEST_SCHEMAS - * @throws Exception - * - can throw any exception, test framework will handle. - * @see DestinationAcceptanceTest.setUpInternal - */ - @Throws(Exception::class) - override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet) { - // use secrets file with GCS staging config - secretsFile = Path.of("secrets/credentials-gcs-staging.json") - setUpBigQuery() - removeOldNamespaces() - - DestinationConfig.initialize(_config) - - // the setup steps below are specific to GCS staging use case - val gcsDestinationConfig: GcsDestinationConfig = - GcsDestinationConfig.getGcsDestinationConfig(getGcsJsonNodeConfig(_config!!)) - this.s3Client = gcsDestinationConfig.getS3Client() - } - - /** - * Removes data from bigquery and GCS This function will be called after EACH test - * - * @param testEnv - * - information about the test environment. - * @throws Exception - * - can throw any exception, test framework will handle. - * @see DestinationAcceptanceTest.tearDownInternal - */ - override fun tearDown(testEnv: TestDestinationEnv) { - tearDownBigQuery() - tearDownGcs() - } - - protected fun tearDownGcs() { - BigQueryDestinationTestUtils.tearDownGcs(s3Client, _config, LOGGER) - } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(BigQueryGcsDestinationAcceptanceTest::class.java) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryStandardDestinationAcceptanceTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryStandardDestinationAcceptanceTest.kt deleted file mode 100644 index 00cbd36d3b982..0000000000000 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryStandardDestinationAcceptanceTest.kt +++ /dev/null @@ -1,50 +0,0 @@ -/* - * Copyright (c) 2023 Airbyte, Inc., all rights reserved. - */ -package io.airbyte.integrations.destination.bigquery - -import java.nio.file.Path -import org.junit.jupiter.api.Disabled -import org.junit.jupiter.api.TestInstance -import org.slf4j.Logger -import org.slf4j.LoggerFactory - -@Disabled -@TestInstance(TestInstance.Lifecycle.PER_CLASS) -class BigQueryStandardDestinationAcceptanceTest : AbstractBigQueryDestinationAcceptanceTest() { - /** - * Sets up secretsFile path and BigQuery instance for verification and cleanup This function - * will be called before EACH test. - * - * @param testEnv - * - information about the test environment. - * @param TEST_SCHEMAS - * @throws Exception - * - can throw any exception, test framework will handle. - * @see DestinationAcceptanceTest.setUpInternal - */ - @Throws(Exception::class) - override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet) { - secretsFile = Path.of("secrets/credentials-standard.json") - setUpBigQuery() - removeOldNamespaces() - } - - /** - * Removes data from bigquery This function will be called after EACH test - * - * @param testEnv - * - information about the test environment. - * @throws Exception - * - can throw any exception, test framework will handle. - * @see DestinationAcceptanceTest.tearDownInternal - */ - override fun tearDown(testEnv: TestDestinationEnv) { - tearDownBigQuery() - } - - companion object { - private val LOGGER: Logger = - LoggerFactory.getLogger(BigQueryStandardDestinationAcceptanceTest::class.java) - } -} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt index ed69c90be8ca0..c6279d41091ea 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt @@ -50,10 +50,11 @@ class LegacyBigQuerySqlGenerator override fun buildColumnId(name: String, suffix: String?): ColumnId { val nameWithSuffix = name + suffix return ColumnId( - nameTransformer.getIdentifier(nameWithSuffix), - name, // Bigquery columns are case-insensitive, so do all our validation on the + nameTransformer.convertStreamName(nameWithSuffix), + name, + // Bigquery columns are case-insensitive, so do all our validation on the // lowercased name - nameTransformer.getIdentifier(nameWithSuffix.lowercase(Locale.getDefault())) + nameTransformer.convertStreamName(nameWithSuffix.lowercase(Locale.getDefault())) ) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt index bd3d538c2af90..4d788a7918544 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestination.kt @@ -3,15 +3,17 @@ */ package io.airbyte.integrations.destination.bigquery -import com.google.cloud.bigquery.BigQueryException import io.airbyte.cdk.AirbyteDestinationRunner -import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addThrowableForDeinterpolation import io.airbyte.cdk.load.command.aws.AwsToolkitConstants +/** + * This is needed because the GCS client is, under the hood, using the S3Client. + * + * And the S3Client depends on the AWS environment - we're not actually _using the assume role + * stuff, but the wiring needs to be satisfied. + */ val additionalMicronautEnvs = listOf(AwsToolkitConstants.MICRONAUT_ENVIRONMENT) fun main(args: Array) { - val additionalMicronautEnvs = listOf(AwsToolkitConstants.MICRONAUT_ENVIRONMENT) - addThrowableForDeinterpolation(BigQueryException::class.java) AirbyteDestinationRunner.run(*args, additionalMicronautEnvs = additionalMicronautEnvs) } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformer.kt index 8309eb6eca8f1..233f1212d2352 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformer.kt @@ -3,11 +3,18 @@ */ package io.airbyte.integrations.destination.bigquery -import io.airbyte.cdk.integrations.destination.StandardNameTransformer +import io.airbyte.cdk.load.data.Transformations -class BigQuerySQLNameTransformer : StandardNameTransformer() { - override fun convertStreamName(input: String): String { - val result = super.convertStreamName(input) +/** + * A legacy class. This used to inherit from the old CDK's StandardNameTransformer. You probably + * should avoid adding new uses of this class. + * + * (I have no explanation for the method names.) + */ +class BigQuerySQLNameTransformer { + /** This seemingly is what we use for any table/column name. */ + fun convertStreamName(input: String): String { + val result = Transformations.toAlphanumericAndUnderscore(input) if (!result.substring(0, 1).matches("[A-Za-z_]".toRegex())) { // has to start with a letter or _ return "_$result" @@ -22,8 +29,8 @@ class BigQuerySQLNameTransformer : StandardNameTransformer() { * normalization. Reference: https://cloud.google.com/bigquery/docs/datasets#dataset-naming * */ - override fun getNamespace(namespace: String): String { - val normalizedName = super.convertStreamName(namespace) + fun getNamespace(namespace: String): String { + val normalizedName = Transformations.toAlphanumericAndUnderscore(namespace) if (!normalizedName.substring(0, 1).matches("[A-Za-z0-9]".toRegex())) { return BigQueryConsts.NAMESPACE_PREFIX + normalizedName } @@ -31,7 +38,7 @@ class BigQuerySQLNameTransformer : StandardNameTransformer() { } @Deprecated("") - override fun getTmpTableName(streamName: String, randomSuffix: String): String { + fun getTmpTableName(streamName: String, randomSuffix: String): String { return convertStreamName("_airbyte_tmp" + "_" + randomSuffix + "_" + streamName) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt index ac898ec8a4620..595de616923ed 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt @@ -42,10 +42,10 @@ class BigqueryFinalTableNameGenerator(val config: BigqueryConfiguration) : Final class BigqueryColumnNameGenerator : ColumnNameGenerator { override fun getColumnName(column: String): ColumnNameGenerator.ColumnName { return ColumnNameGenerator.ColumnName( - nameTransformer.getIdentifier(column), + nameTransformer.convertStreamName(column), // Bigquery columns are case-insensitive, so do all our validation on the // lowercased name - nameTransformer.getIdentifier(column.lowercase(Locale.getDefault())), + nameTransformer.convertStreamName(column.lowercase(Locale.getDefault())), ) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformerTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformerTest.kt index 00e394cfa709d..8dfb900cb0792 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformerTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/BigQuerySQLNameTransformerTest.kt @@ -7,14 +7,6 @@ import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Test internal class BigQuerySQLNameTransformerTest { - @Test - fun testGetIdentifier() { - RAW_TO_NORMALIZED_IDENTIFIERS.forEach { (raw: String?, normalized: String?) -> - Assertions.assertEquals(normalized, INSTANCE.getIdentifier(raw)) - Assertions.assertEquals(normalized, INSTANCE.convertStreamName(raw)) - } - } - @Test fun testGetNamespace() { RAW_TO_NORMALIZED_NAMESPACES.forEach { (raw: String?, normalized: String?) -> @@ -24,19 +16,6 @@ internal class BigQuerySQLNameTransformerTest { companion object { private val INSTANCE = BigQuerySQLNameTransformer() - private val RAW_TO_NORMALIZED_IDENTIFIERS: Map = - java.util.Map.of( - "name-space", - "name_space", - "spécial_character", - "special_character", - "99namespace", - "_99namespace", - "*_namespace", - "__namespace", - "_namespace", - "_namespace" - ) private val RAW_TO_NORMALIZED_NAMESPACES: Map = java.util.Map.of( From a3fa044592fad2e45deb76e181f21b6b6e8ce8ff Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:39:27 -0700 Subject: [PATCH 30/55] column name stuff --- .../cdk/load/message/DestinationMessage.kt | 3 + .../destination/bigquery/BigQueryUtils.kt | 74 +++---------------- .../formatter/BigQueryRecordFormatter.kt | 43 ++--------- ...igqueryDestinationInitialStatusGatherer.kt | 4 +- .../write/BigqueryRawTableOperations.kt | 4 +- 5 files changed, 26 insertions(+), 102 deletions(-) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt index add68f2d8c025..5ca7c50b9d6ab 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/message/DestinationMessage.kt @@ -126,6 +126,9 @@ data class Meta( COLUMN_NAME_AB_GENERATION_ID, ) + /** A legacy column name. Used in "DV2" destinations' raw tables. */ + const val COLUMN_NAME_AB_LOADED_AT = "_airbyte_loaded_at" + fun getMetaValue(metaColumnName: String, value: String): AirbyteValue { if (!COLUMN_NAMES.contains(metaColumnName)) { throw IllegalArgumentException("Invalid meta column name: $metaColumnName") diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt index bd7cc18a0694a..6a3e297ae2674 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt @@ -9,16 +9,12 @@ import com.google.cloud.RetryOption import com.google.cloud.bigquery.* import com.google.common.collect.ImmutableList import com.google.common.collect.ImmutableMap -import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig -import io.airbyte.commons.json.Jsons.deserialize -import io.airbyte.commons.json.Jsons.jsonNode +import io.airbyte.cdk.load.message.Meta +import io.airbyte.cdk.load.util.Jsons import java.util.* import java.util.stream.Collectors import org.apache.commons.lang3.StringUtils import org.apache.commons.lang3.tuple.ImmutablePair -import org.apache.logging.log4j.util.Strings import org.slf4j.Logger import org.slf4j.LoggerFactory import org.threeten.bp.Duration @@ -56,7 +52,6 @@ object BigQueryUtils { fun waitForQuery(queryJob: Job): Job? { try { val job = queryJob.waitFor() - AirbyteExceptionHandler.addStringForDeinterpolation(job.etag) return job } catch (e: Exception) { LOGGER.error("Failed to wait for a query job:$queryJob") @@ -94,12 +89,12 @@ object BigQueryUtils { try { val partitioning = TimePartitioning.newBuilder(TimePartitioning.Type.DAY) - .setField(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT) + .setField(Meta.COLUMN_NAME_AB_EXTRACTED_AT) .build() val clustering = Clustering.newBuilder() - .setFields(ImmutableList.of(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT)) + .setFields(ImmutableList.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT)) .build() val tableDefinition = @@ -126,7 +121,7 @@ object BigQueryUtils { @JvmStatic fun getGcsJsonNodeConfig(config: JsonNode): JsonNode { val loadingMethod = config[BigQueryConsts.LOADING_METHOD] - return jsonNode( + return Jsons.valueToTree( ImmutableMap.builder() .put(BigQueryConsts.GCS_BUCKET_NAME, loadingMethod[BigQueryConsts.GCS_BUCKET_NAME]) .put(BigQueryConsts.GCS_BUCKET_PATH, loadingMethod[BigQueryConsts.GCS_BUCKET_PATH]) @@ -134,19 +129,17 @@ object BigQueryUtils { .put(BigQueryConsts.CREDENTIAL, loadingMethod[BigQueryConsts.CREDENTIAL]) .put( BigQueryConsts.FORMAT, - deserialize("""{ - "format_type": "CSV", - "flattening": "No flattening" -}""") + Jsons.readTree( + """{ + "format_type": "CSV", + "flattening": "No flattening" + }""" + ) ) .build() ) } - fun getGcsCsvDestinationConfig(config: JsonNode): GcsDestinationConfig { - return GcsDestinationConfig.getGcsDestinationConfig(getGcsJsonNodeConfig(config)) - } - /** @return a default schema name based on the config. */ @JvmStatic fun getDatasetId(config: JsonNode): String { @@ -178,32 +171,6 @@ object BigQueryUtils { } } - fun getDisableTypeDedupFlag(config: JsonNode): Boolean { - if (config.has(BigQueryConsts.DISABLE_TYPE_DEDUPE)) { - return config[BigQueryConsts.DISABLE_TYPE_DEDUPE].asBoolean(false) - } - - return false - } - - // https://googleapis.dev/python/bigquery/latest/generated/google.cloud.bigquery.client.Client.html - fun getBigQueryClientChunkSize(config: JsonNode): Int? { - var chunkSizeFromConfig: Int? = null - if (config.has(BigQueryConsts.BIG_QUERY_CLIENT_CHUNK_SIZE)) { - chunkSizeFromConfig = config[BigQueryConsts.BIG_QUERY_CLIENT_CHUNK_SIZE].asInt() - if (chunkSizeFromConfig <= 0) { - LOGGER.error( - "BigQuery client Chunk (buffer) size must be a positive number (MB), but was:$chunkSizeFromConfig" - ) - throw IllegalArgumentException( - "BigQuery client Chunk (buffer) size must be a positive number (MB)" - ) - } - chunkSizeFromConfig = chunkSizeFromConfig * BigQueryConsts.MiB - } - return chunkSizeFromConfig - } - @JvmStatic fun getLoadingMethod(config: JsonNode): UploadingMethod { val loadingMethod = config[BigQueryConsts.LOADING_METHOD] @@ -219,26 +186,9 @@ object BigQueryUtils { } } - fun isKeepFilesInGcs(config: JsonNode): Boolean { - val loadingMethod = config[BigQueryConsts.LOADING_METHOD] - if ( - loadingMethod != null && - loadingMethod[BigQueryConsts.KEEP_GCS_FILES] != null && - (BigQueryConsts.KEEP_GCS_FILES_VAL == - loadingMethod[BigQueryConsts.KEEP_GCS_FILES].asText()) - ) { - LOGGER.info("All tmp files GCS will be kept in bucket when replication is finished") - return true - } else { - LOGGER.info("All tmp files will be removed from GCS when replication is finished") - return false - } - } - @Throws(InterruptedException::class) fun waitForJobFinish(job: Job?) { if (job != null) { - AirbyteExceptionHandler.addStringForDeinterpolation(job.etag) try { LOGGER.info("Waiting for Job {} to finish. Status: {}", job.jobId, job.status) // Default totalTimeout is 12 Hours, 30 minutes seems reasonable @@ -302,6 +252,6 @@ object BigQueryUtils { private val connectorNameOrDefault: String get() = Optional.ofNullable(System.getenv("WORKER_CONNECTOR_IMAGE")) - .map { name: String -> name.replace("airbyte/", Strings.EMPTY).replace(":", "/") } + .map { name: String -> name.replace("airbyte/", "").replace(":", "/") } .orElse("destination-bigquery") } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt index 9ebc2b00dc126..6899485b0e404 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt @@ -3,25 +3,16 @@ */ package io.airbyte.integrations.destination.bigquery.formatter -import com.fasterxml.jackson.databind.JsonNode -import com.fasterxml.jackson.databind.node.NullNode -import com.fasterxml.jackson.databind.node.ObjectNode import com.google.cloud.bigquery.Field import com.google.cloud.bigquery.QueryParameterValue import com.google.cloud.bigquery.Schema import com.google.cloud.bigquery.StandardSQLTypeName -import io.airbyte.cdk.integrations.base.JavaBaseConstants -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage -import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage import io.airbyte.cdk.load.data.IntegerValue import io.airbyte.cdk.load.data.ObjectValue import io.airbyte.cdk.load.data.StringValue import io.airbyte.cdk.load.message.DestinationRecordRaw import io.airbyte.cdk.load.message.Meta import io.airbyte.cdk.load.util.serializeToString -import io.airbyte.commons.json.Jsons.emptyObject -import io.airbyte.commons.json.Jsons.serialize -import java.util.* import java.util.concurrent.TimeUnit /** @@ -29,19 +20,6 @@ import java.util.concurrent.TimeUnit * corresponding uploader. */ class BigQueryRecordFormatter { - fun formatRecord(recordMessage: PartialAirbyteMessage, generationId: Long): String { - val record = emptyObject() as ObjectNode - record.put(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, UUID.randomUUID().toString()) - record.put( - JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, - getEmittedAtField(recordMessage.record) - ) - record.set(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, NullNode.instance) - record.put(JavaBaseConstants.COLUMN_NAME_DATA, recordMessage.serialized) - record.put(JavaBaseConstants.COLUMN_NAME_AB_META, serialize(recordMessage.record!!.meta!!)) - record.put(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, generationId) - return serialize(record) - } fun formatRecord(record: DestinationRecordRaw): String { val enrichedRecord = record.asEnrichedDestinationRecordAirbyteValue() @@ -64,15 +42,11 @@ class BigQueryRecordFormatter { } } - outputRecord[JavaBaseConstants.COLUMN_NAME_DATA] = record.asRawJson().serializeToString() + outputRecord[Meta.COLUMN_NAME_DATA] = record.asRawJson().serializeToString() return outputRecord.serializeToString() } - private fun getEmittedAtField(recordMessage: PartialAirbyteRecordMessage?): String? { - return getExtractedAt(recordMessage!!.emittedAt) - } - private fun getExtractedAt(extractedAtMillis: Long): String? { // Bigquery represents TIMESTAMP to the microsecond precision, so we convert to microseconds // then use BQ helpers to string-format correctly. @@ -85,15 +59,12 @@ class BigQueryRecordFormatter { // This is the schema used to represent the final raw table val SCHEMA_V2: Schema = Schema.of( - Field.of(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING), - Field.of( - JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, - StandardSQLTypeName.TIMESTAMP - ), - Field.of(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, StandardSQLTypeName.TIMESTAMP), - Field.of(JavaBaseConstants.COLUMN_NAME_DATA, StandardSQLTypeName.STRING), - Field.of(JavaBaseConstants.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING), - Field.of(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64) + Field.of(Meta.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP), + Field.of(Meta.COLUMN_NAME_AB_LOADED_AT, StandardSQLTypeName.TIMESTAMP), + Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64) ) // This schema defines the CSV format used for the load job. It differs from SCHEMA_V2 by diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index 8ce0ea5eae605..c07780e4e5b67 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -13,12 +13,12 @@ import com.google.cloud.bigquery.TableDefinition import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.TimePartitioning import com.google.common.annotations.VisibleForTesting -import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.load.command.Append import io.airbyte.cdk.load.command.Dedupe import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.Overwrite import io.airbyte.cdk.load.data.ObjectType +import io.airbyte.cdk.load.message.Meta import io.airbyte.cdk.load.orchestration.ColumnNameMapping import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.TableName @@ -197,7 +197,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : .stream() .filter { name: String -> !containsIgnoreCase(streamSchema.keys, name) && - !containsIgnoreCase(JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS, name) + !containsIgnoreCase(Meta.COLUMN_NAMES, name) } .collect(Collectors.toSet()) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt index 30fd0e151ab6d..98959c2977c71 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt @@ -8,7 +8,7 @@ import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.QueryJobConfiguration import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.TableResult -import io.airbyte.cdk.integrations.base.JavaBaseConstants +import io.airbyte.cdk.load.message.Meta import io.airbyte.cdk.load.orchestration.TableName import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingRawTableOperations import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -79,7 +79,7 @@ class BigqueryRawTableOperations(private val bigquery: BigQuery) : if (result.totalRows == 0L) { return null } - val value = result.iterateAll().first().get(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID) + val value = result.iterateAll().first().get(Meta.COLUMN_NAME_AB_GENERATION_ID) return if (value == null || value.isNull) { 0 } else { From e148eca0c9f9e027133225e7165cb8d4d2cabf69 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:42:27 -0700 Subject: [PATCH 31/55] last stuff? --- .../kotlin/io/airbyte/cdk/util/ConnectorExceptionUtil.kt | 9 +++++++++ .../typing_deduping/BigQueryDestinationHandler.kt | 6 ++---- 2 files changed, 11 insertions(+), 4 deletions(-) create mode 100644 airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/ConnectorExceptionUtil.kt diff --git a/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/ConnectorExceptionUtil.kt b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/ConnectorExceptionUtil.kt new file mode 100644 index 0000000000000..303c26a8841d6 --- /dev/null +++ b/airbyte-cdk/bulk/core/base/src/main/kotlin/io/airbyte/cdk/util/ConnectorExceptionUtil.kt @@ -0,0 +1,9 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.util + +object ConnectorExceptionUtil { + val HTTP_AUTHENTICATION_ERROR_CODES: List = listOf(401, 403) +} diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt index cdf80efc5a42d..fc9f40dc70baa 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt @@ -13,11 +13,10 @@ import com.google.cloud.bigquery.JobStatistics import com.google.cloud.bigquery.JobStatus import com.google.cloud.bigquery.QueryJobConfiguration import com.google.common.collect.Streams -import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler -import io.airbyte.cdk.integrations.util.ConnectorExceptionUtil +import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.orchestration.DestinationHandler import io.airbyte.cdk.load.orchestration.Sql -import io.airbyte.commons.exceptions.ConfigErrorException +import io.airbyte.cdk.util.ConnectorExceptionUtil import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.github.oshai.kotlinlogging.KotlinLogging import java.util.UUID @@ -51,7 +50,6 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo QueryJobConfiguration.newBuilder(statement).build() ) ) - AirbyteExceptionHandler.addStringForDeinterpolation(job.etag) // job.waitFor() gets stuck forever in some failure cases, so manually poll the job instead. while (JobStatus.State.DONE != job.status.state) { Thread.sleep(1000L) From 298d9bfe6aa131d867095a21322c71c415388fd3 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:43:31 -0700 Subject: [PATCH 32/55] kill old CDK deps from runtime code --- .../connectors/destination-bigquery/build.gradle | 9 --------- 1 file changed, 9 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/build.gradle b/airbyte-integrations/connectors/destination-bigquery/build.gradle index 3d772df61c3aa..e7a239d57e62b 100644 --- a/airbyte-integrations/connectors/destination-bigquery/build.gradle +++ b/airbyte-integrations/connectors/destination-bigquery/build.gradle @@ -32,19 +32,10 @@ application { } dependencies { - implementation 'org.apache.commons:commons-text:1.12.0' implementation 'com.google.cloud:google-cloud-bigquery' integrationTestImplementation 'com.google.cloud:google-cloud-bigquery' - // old CDK dependencies - implementation 'io.airbyte.cdk:airbyte-cdk-core:0.46.0' - implementation 'io.airbyte.cdk:airbyte-cdk-datastore-bigquery:0.46.0' - implementation 'io.airbyte.cdk:airbyte-cdk-db-destinations:0.46.0' - implementation 'io.airbyte.cdk:airbyte-cdk-dependencies:0.46.0' - implementation 'io.airbyte.cdk:airbyte-cdk-gcs-destinations:0.46.0' - implementation 'io.airbyte.cdk:airbyte-cdk-typing-deduping:0.46.0' - integrationTestLegacyImplementation 'io.airbyte.cdk:airbyte-cdk-core:0.46.0' integrationTestLegacyImplementation 'io.airbyte.cdk:airbyte-cdk-datastore-bigquery:0.46.0' integrationTestLegacyImplementation 'io.airbyte.cdk:airbyte-cdk-db-destinations:0.46.0' From 9cf1fb3a38f904b1d91ad68cba38c3cc671f93c2 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 08:57:47 -0700 Subject: [PATCH 33/55] clean up file --- .../bigquery/BigQueryDestinationTest.kt | 229 +----------------- 1 file changed, 2 insertions(+), 227 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt index c526df44ec129..ea43cf00c88ae 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryDestinationTest.kt @@ -11,11 +11,9 @@ import com.google.common.collect.Lists import io.airbyte.cdk.integrations.base.DestinationConfig import io.airbyte.cdk.integrations.base.JavaBaseConstants import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig -import io.airbyte.commons.json.Jsons.deserialize import io.airbyte.commons.json.Jsons.jsonNode import io.airbyte.commons.string.Strings.addRandomSuffix import io.airbyte.integrations.destination.bigquery.BigQueryUtils.createPartitionedTableIfNotExists -import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetLocation import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getGcsJsonNodeConfig import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getOrCreateDataset @@ -27,14 +25,7 @@ import java.nio.file.Files import java.nio.file.Path import java.time.Instant import java.util.* -import java.util.function.Function -import java.util.stream.Collectors -import java.util.stream.Stream -import java.util.stream.StreamSupport import org.junit.jupiter.api.* -import org.junit.jupiter.params.ParameterizedTest -import org.junit.jupiter.params.provider.Arguments -import org.junit.jupiter.params.provider.MethodSource import org.mockito.Mockito.mock import org.slf4j.Logger import org.slf4j.LoggerFactory @@ -45,48 +36,6 @@ internal class BigQueryDestinationTest { protected var dataset: Dataset? = null private var s3Client: AmazonS3? = null - private fun successTestConfigProviderBase(): Stream { - return Stream.of(Arguments.of("config"), Arguments.of("configWithProjectId")) - } - - private fun successTestConfigProvider(): Stream { - return Stream.concat( - successTestConfigProviderBase(), - Stream.of(Arguments.of("gcsStagingConfig")) - ) - } - - private fun failCheckTestConfigProvider(): Stream { - return Stream.of( - Arguments.of( - "configWithBadProjectId", - "User does not have bigquery.datasets.create permission in project" - ), - Arguments.of( - "insufficientRoleConfig", - "User does not have bigquery.datasets.create permission" - ), - Arguments.of( - "gcsStagingConfigWithBadCopyPermission", - "Permission bigquery.tables.updateData denied on table" - ), - ) - } - - private fun failWriteTestConfigProvider(): Stream { - return Stream.of( - Arguments.of( - "configWithBadProjectId", - "User does not have bigquery.datasets.create permission in project" - ), - Arguments.of( - "noEditPublicSchemaRoleConfig", - "Failed to write to destination schema." - ), // (or it may not exist) - Arguments.of("insufficientRoleConfig", "Permission bigquery.tables.create denied") - ) - } - @Throws(IOException::class) protected fun initBigQuery(config: JsonNode) { bigquery = BigQueryDestinationTestUtils.initBigQuery(config) @@ -119,25 +68,6 @@ internal class BigQueryDestinationTest { BigQueryDestinationTestUtils.tearDownGcs(s3Client, config, LOGGER) } - @Disabled @Test @Throws(Exception::class) fun testSpec() {} - - @Disabled - @ParameterizedTest - @MethodSource("successTestConfigProvider") - @Throws(IOException::class) - fun testCheckSuccess(configName: String) {} - - @Disabled - @ParameterizedTest - @MethodSource("failCheckTestConfigProvider") - fun testCheckFailures(configName: String, error: String?) {} - - @Disabled - @ParameterizedTest - @MethodSource("successTestConfigProvider") - @Throws(Exception::class) - fun testWriteSuccess(configName: String) {} - @Test @Throws(Exception::class) fun testCreateTableSuccessWhenTableAlreadyExists() { @@ -164,9 +94,9 @@ internal class BigQueryDestinationTest { val tableId = TableId.of(tmpTestSchemaName, "test_already_existing_table") - getOrCreateDataset(bigquery!!, tmpTestSchemaName, getDatasetLocation(config!!)) + getOrCreateDataset(bigquery!!, tmpTestSchemaName, getDatasetLocation(config)) - org.junit.jupiter.api.Assertions.assertDoesNotThrow { + Assertions.assertDoesNotThrow { // Create table createPartitionedTableIfNotExists(bigquery!!, tableId, schema) @@ -175,150 +105,6 @@ internal class BigQueryDestinationTest { } } - @Disabled - @ParameterizedTest - @MethodSource("failWriteTestConfigProvider") - @Throws(Exception::class) - fun testWriteFailure(configName: String, error: String?) {} - - @Throws(InterruptedException::class) - private fun fetchNamesOfTablesInDb(): Set { - if (dataset == null || bigquery == null) { - return emptySet() - } - val queryConfig = - QueryJobConfiguration.newBuilder( - String.format( - "SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES`;", - dataset!!.datasetId.dataset - ) - ) - .setUseLegacySql(false) - .build() - - if (!dataset!!.exists()) { - return emptySet() - } - return StreamSupport.stream( - executeQuery(bigquery!!, queryConfig) - .getLeft() - .getQueryResults() - .iterateAll() - .spliterator(), - false - ) - .map { v: FieldValueList -> v["TABLE_NAME"].stringValue } - .collect(Collectors.toSet()) - } - - @Throws(InterruptedException::class) - private fun assertTmpTablesNotPresent(tableNames: List) { - val tmpTableNamePrefixes = - tableNames.stream().map { name: String -> name + "_" }.collect(Collectors.toSet()) - val finalTableNames = - tableNames.stream().map { name: String -> name + "_raw" }.collect(Collectors.toSet()) - // search for table names that have the tmp table prefix but are not raw tables. - org.junit.jupiter.api.Assertions.assertTrue( - fetchNamesOfTablesInDb() - .stream() - .filter { tableName: String -> !finalTableNames.contains(tableName) } - .noneMatch { tableName: String -> - tmpTableNamePrefixes.stream().anyMatch { prefix: String -> - tableName.startsWith(prefix) - } - } - ) - } - - @Throws(Exception::class) - private fun retrieveRecords(tableName: String): List { - val queryConfig = - QueryJobConfiguration.newBuilder( - String.format( - "SELECT * FROM `%s.%s`;", - dataset!!.datasetId.dataset, - tableName.lowercase(Locale.getDefault()) - ) - ) - .setUseLegacySql(false) - .build() - - executeQuery(bigquery!!, queryConfig) - - return StreamSupport.stream( - executeQuery(bigquery!!, queryConfig) - .getLeft() - .getQueryResults() - .iterateAll() - .spliterator(), - false - ) - .map( - Function { v: FieldValueList -> - v.get(JavaBaseConstants.COLUMN_NAME_DATA).getStringValue() - } - ) - .map { jsonString: String? -> deserialize(jsonString) } - .collect(Collectors.toList()) - } - - @Disabled - @ParameterizedTest - @MethodSource("successTestConfigProviderBase") - @Throws(Exception::class) - fun testWritePartitionOverUnpartitioned(configName: String) {} - - private fun createUnpartitionedTable( - bigquery: BigQuery?, - dataset: Dataset?, - tableName: String - ) { - val tableId = TableId.of(dataset!!.datasetId.dataset, tableName) - bigquery!!.delete(tableId) - val schema = - Schema.of( - com.google.cloud.bigquery.Field.of( - JavaBaseConstants.COLUMN_NAME_AB_ID, - StandardSQLTypeName.STRING - ), - com.google.cloud.bigquery.Field.of( - JavaBaseConstants.COLUMN_NAME_EMITTED_AT, - StandardSQLTypeName.TIMESTAMP - ), - com.google.cloud.bigquery.Field.of( - JavaBaseConstants.COLUMN_NAME_DATA, - StandardSQLTypeName.STRING - ) - ) - val tableDefinition = StandardTableDefinition.newBuilder().setSchema(schema).build() - val tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build() - bigquery.create(tableInfo) - } - - @Throws(InterruptedException::class) - private fun isTablePartitioned( - bigquery: BigQuery?, - dataset: Dataset?, - tableName: String - ): Boolean { - val queryConfig = - QueryJobConfiguration.newBuilder( - String.format( - "SELECT max(is_partitioning_column) as is_partitioned FROM `%s.%s.INFORMATION_SCHEMA.COLUMNS` WHERE TABLE_NAME = '%s';", - bigquery!!.options.projectId, - dataset!!.datasetId.dataset, - tableName - ) - ) - .setUseLegacySql(false) - .build() - val result = executeQuery(bigquery, queryConfig) - for (row in result.getLeft().getQueryResults().values) { - return !row["is_partitioned"].isNull && row["is_partitioned"].stringValue == "YES" - } - return false - } - companion object { protected val CREDENTIALS_STANDARD_INSERT_PATH: Path = Path.of("secrets/credentials-standard.json") @@ -414,17 +200,6 @@ internal class BigQueryDestinationTest { ) .withEmittedAt(NOW.toEpochMilli()) ) - protected val MESSAGE_STATE: AirbyteMessage = - AirbyteMessage() - .withType(AirbyteMessage.Type.STATE) - .withState( - AirbyteStateMessage() - .withData( - jsonNode( - ImmutableMap.builder().put("checkpoint", "now!").build() - ) - ) - ) protected var projectId: String? = null protected var datasetId: String? = null From 16924938b9dfbdf539cbcd24359982a70a0e66f6 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 09:06:19 -0700 Subject: [PATCH 34/55] derp --- .../bigquery/typing_deduping/BigQuerySqlGenerator.kt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index bd90365924916..161fbfb76ef98 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -643,9 +643,9 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo ArrayTypeWithoutSchema, is ObjectType, ObjectTypeWithEmptySchema, - ObjectTypeWithoutSchema, + ObjectTypeWithoutSchema -> StandardSQLTypeName.JSON is UnionType -> - if ((type as UnionType).isLegacyUnion) { + if (type.isLegacyUnion) { toDialectType(type.chooseType()) } else { StandardSQLTypeName.JSON From d8401ff0575ac457c22a02ab52d46effa2670898 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Mon, 21 Apr 2025 15:45:13 -0700 Subject: [PATCH 35/55] uniqify --- .../io/airbyte/cdk/load/orchestration/DestinationHandler.kt | 2 +- .../legacy_typing_deduping/TypingDedupingWriter.kt | 4 ++-- .../bigquery/typing_deduping/BigQueryDestinationHandler.kt | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt index de966e5a70402..047dfd225cf78 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt @@ -13,5 +13,5 @@ interface DestinationHandler { * This function should assume that all `namespaces` are valid identifiers, i.e. any special * characters have already been escaped, they respect identifier name length, etc. */ - suspend fun createNamespaces(namespaces: List) + suspend fun createNamespaces(namespaces: Collection) } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt index a1e25761dc18b..b6b9544a0e121 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt @@ -30,10 +30,10 @@ class TypingDedupingWriter( override suspend fun setup() { Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> - destinationHandler.createNamespaces( + val namespaces = names.values.map { (tableNames, _) -> tableNames.rawTableName!!.namespace } + names.values.map { (tableNames, _) -> tableNames.finalTableName!!.namespace } - ) + destinationHandler.createNamespaces(namespaces.toSet()) val initialInitialStatuses: Map = diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt index fc9f40dc70baa..0a7860410f94c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt @@ -110,7 +110,7 @@ class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLo } } - override suspend fun createNamespaces(namespaces: List) { + override suspend fun createNamespaces(namespaces: Collection) { coroutineScope { namespaces.forEach { dataset -> launch { From dddb3f586183ab3ed92fe2bc97b2ec00e6088946 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 08:57:45 -0700 Subject: [PATCH 36/55] move to subpackage --- .../{ => db}/DestinationHandler.kt | 2 +- .../DestinationInitialStatusGatherer.kt | 4 +- .../{ => db}/DestinationNames.kt | 2 +- .../cdk/load/orchestration/{ => db}/Sql.kt | 2 +- .../DirectLoadSqlGenerator.kt | 2 +- .../DirectLoadTableOperations.kt | 6 +-- .../DirectLoadTableStreamLoader.kt | 2 +- .../DirectLoadTableWriter.kt | 2 +- .../AlterTableReport.kt | 2 +- .../TableNameMapFactory.kt | 22 ++++----- .../TypingDedupingDestinationInitialStatus.kt | 10 ++-- .../TypingDedupingFinalTableOperations.kt | 48 +++++++++---------- .../TypingDedupingRawTableOperations.kt | 4 +- .../TypingDedupingSqlGenerator.kt | 46 +++++++++--------- .../TypingDedupingStreamLoader.kt | 30 ++++++------ .../TypingDedupingUtil.kt | 2 +- .../TypingDedupingWriter.kt | 24 +++++----- .../BigQueryDestinationHandler.kt | 4 +- .../typing_deduping/BigQuerySqlGenerator.kt | 12 ++--- ...igqueryDestinationInitialStatusGatherer.kt | 40 ++++++++-------- .../typing_deduping/BigqueryNameGenerators.kt | 10 ++-- .../write/BigqueryRawTableOperations.kt | 6 +-- .../bigquery/write/BigqueryWriter.kt | 8 ++-- .../write/bulk_loader/BigQueryBulkLoader.kt | 6 +-- .../BigqueryBatchStandardInsertLoader.kt | 4 +- 25 files changed, 150 insertions(+), 150 deletions(-) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/DestinationHandler.kt (91%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/DestinationInitialStatusGatherer.kt (86%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/DestinationNames.kt (98%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/Sql.kt (98%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/direct_load_table/DirectLoadSqlGenerator.kt (62%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/direct_load_table/DirectLoadTableOperations.kt (82%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/direct_load_table/DirectLoadTableStreamLoader.kt (91%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/direct_load_table/DirectLoadTableWriter.kt (88%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/AlterTableReport.kt (87%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TableNameMapFactory.kt (65%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt (84%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt (79%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingRawTableOperations.kt (93%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingSqlGenerator.kt (79%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingStreamLoader.kt (95%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingUtil.kt (93%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/{ => db}/legacy_typing_deduping/TypingDedupingWriter.kt (80%) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt similarity index 91% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt index 047dfd225cf78..a9d03382b49f6 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationHandler.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration +package io.airbyte.cdk.load.orchestration.db interface DestinationHandler { fun execute(sql: Sql) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt similarity index 86% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt index b97ad211673f9..f75c3489dbc6a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationInitialStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration +package io.airbyte.cdk.load.orchestration.db import io.airbyte.cdk.load.command.DestinationStream @@ -24,6 +24,6 @@ interface DestinationInitialStatus */ fun interface DestinationInitialStatusGatherer { suspend fun gatherInitialStatus( - streams: Map>, + streams: Map>, ): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt similarity index 98% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt index 3589f2383026a..9cb6f2067473a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration +package io.airbyte.cdk.load.orchestration.db import io.airbyte.cdk.load.command.DestinationStream diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt similarity index 98% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt index c857d81dcd092..25ca51c1d7c1d 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/Sql.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt @@ -1,4 +1,4 @@ -package io.airbyte.cdk.load.orchestration +package io.airbyte.cdk.load.orchestration.db import java.util.function.Consumer diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadSqlGenerator.kt similarity index 62% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadSqlGenerator.kt index 5be1868b1d58f..1f4a9f66bbc1e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadSqlGenerator.kt @@ -2,6 +2,6 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.direct_load_table +package io.airbyte.cdk.load.orchestration.db.direct_load_table interface DirectLoadSqlGenerator {} diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt similarity index 82% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt index ae05c607f8c4a..c948995d73c79 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt @@ -2,11 +2,11 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.direct_load_table +package io.airbyte.cdk.load.orchestration.db.direct_load_table import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.DestinationHandler -import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.TableName class DirectLoadTableOperations( private val sqlGenerator: DirectLoadSqlGenerator, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableStreamLoader.kt similarity index 91% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableStreamLoader.kt index fc779019caf8f..b7bf3ed4da009 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableStreamLoader.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.direct_load_table +package io.airbyte.cdk.load.orchestration.db.direct_load_table import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.state.StreamProcessingFailed diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableWriter.kt similarity index 88% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableWriter.kt index 6a266310148fa..eb5d95eee2b26 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/direct_load_table/DirectLoadTableWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableWriter.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.direct_load_table +package io.airbyte.cdk.load.orchestration.db.direct_load_table import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.write.DestinationWriter diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/AlterTableReport.kt similarity index 87% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/AlterTableReport.kt index 70dcba5fbfdae..5cfc5c36b92d9 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/AlterTableReport.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/AlterTableReport.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping data class AlterTableReport( val columnsToAdd: Set, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt similarity index 65% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt index 65bf1ab9d0d92..8f309c62ec792 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TableNameMapFactory.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt @@ -2,24 +2,24 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameGenerator -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.FinalTableNameGenerator -import io.airbyte.cdk.load.orchestration.RawTableNameGenerator -import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.db.ColumnNameGenerator +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.FinalTableNameGenerator +import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator +import io.airbyte.cdk.load.orchestration.db.TableNames import io.micronaut.context.annotation.Factory import javax.inject.Singleton @Factory class TypingDedupingTableNameMapFactory( - private val catalog: DestinationCatalog, - private val rawTableNameGenerator: RawTableNameGenerator, - private val finalTableNameGenerator: FinalTableNameGenerator, - private val finalTableColumnNameGenerator: ColumnNameGenerator, + private val catalog: DestinationCatalog, + private val rawTableNameGenerator: RawTableNameGenerator, + private val finalTableNameGenerator: FinalTableNameGenerator, + private val finalTableColumnNameGenerator: ColumnNameGenerator, ) { @Singleton fun get(): Map> { @@ -43,7 +43,7 @@ class TypingDedupingTableNameMapFactory( @Factory class TypingDedupingTableNameMapByDescriptorFactory( - private val map: Map>, + private val map: Map>, ) { @Singleton fun get(): Map> { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt similarity index 84% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt index 3fd5cd7fc8df6..971bc06e51f42 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -2,16 +2,16 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping -import io.airbyte.cdk.load.orchestration.DestinationInitialStatus +import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatus import java.time.Instant data class TypingDedupingDestinationInitialStatus( - /** Initial status of the final table, or null if the table doesn't exist yet. */ + /** Initial status of the final table, or null if the table doesn't exist yet. */ val finalTableStatus: FinalTableInitialStatus?, - val rawTableStatus: RawTableInitialStatus?, - val tempRawTableStatus: RawTableInitialStatus?, + val rawTableStatus: RawTableInitialStatus?, + val tempRawTableStatus: RawTableInitialStatus?, ) : DestinationInitialStatus data class FinalTableInitialStatus( diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt similarity index 79% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 58c7dbcd495dc..3e78ea59bf094 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -2,29 +2,29 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.DestinationHandler -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.TableNames.Companion.SOFT_RESET_SUFFIX +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.SOFT_RESET_SUFFIX import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant private val logger = KotlinLogging.logger {} class TypingDedupingFinalTableOperations( - private val sqlGenerator: TypingDedupingSqlGenerator, - private val destinationHandler: DestinationHandler, + private val sqlGenerator: TypingDedupingSqlGenerator, + private val destinationHandler: DestinationHandler, ) { fun createFinalTable( - stream: DestinationStream, - finalTableName: TableName, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - replace: Boolean + stream: DestinationStream, + finalTableName: TableName, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + replace: Boolean ) { logger.info { "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()}" @@ -42,9 +42,9 @@ class TypingDedupingFinalTableOperations( /** Reset the final table using a temp table or ALTER existing table's columns. */ fun softResetFinalTable( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, ) { logger.info { "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" @@ -73,9 +73,9 @@ class TypingDedupingFinalTableOperations( * specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE ... SELECT *, DROP TABLE */ fun overwriteFinalTable( - stream: DestinationStream, - finalTableName: TableName, - finalTableSuffix: String, + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String, ) { logger.info { "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" @@ -90,11 +90,11 @@ class TypingDedupingFinalTableOperations( } fun typeAndDedupe( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, - maxProcessedTimestamp: Instant?, - finalTableSuffix: String + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + maxProcessedTimestamp: Instant?, + finalTableSuffix: String ) { try { logger.info { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingRawTableOperations.kt similarity index 93% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingRawTableOperations.kt index a8886585ad12f..b515c56267e95 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingRawTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingRawTableOperations.kt @@ -2,9 +2,9 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping -import io.airbyte.cdk.load.orchestration.TableName +import io.airbyte.cdk.load.orchestration.db.TableName interface TypingDedupingRawTableOperations { /** diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt similarity index 79% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index 288b5f105d1b9..43a957a0cc7d2 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -2,14 +2,14 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.Sql -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.TableNames.Companion.SOFT_RESET_SUFFIX +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.Sql +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.SOFT_RESET_SUFFIX import java.time.Instant interface TypingDedupingSqlGenerator { @@ -26,11 +26,11 @@ interface TypingDedupingSqlGenerator { * this to true. */ fun createFinalTable( - stream: DestinationStream, - tableName: TableName, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - replace: Boolean + stream: DestinationStream, + tableName: TableName, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + replace: Boolean ): Sql /** @@ -69,12 +69,12 @@ interface TypingDedupingSqlGenerator { * which handles casting exceptions. */ fun updateFinalTable( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - maxProcessedTimestamp: Instant?, - useExpensiveSaferCasting: Boolean, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + maxProcessedTimestamp: Instant?, + useExpensiveSaferCasting: Boolean, ): Sql /** @@ -84,18 +84,18 @@ interface TypingDedupingSqlGenerator { * non-empty. Callers are responsible for verifying those are true. */ fun overwriteFinalTable( - stream: DestinationStream, - finalTableName: TableName, - finalTableSuffix: String, + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String, ): Sql fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql /** Typically we need to create a soft reset temporary table and clear loaded at values */ fun prepareTablesForSoftReset( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, ): Sql { val createTempTable = createFinalTable( diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt similarity index 95% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 08804ecca00cf..007e48035e77e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -2,13 +2,13 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.TableNames.Companion.NO_SUFFIX -import io.airbyte.cdk.load.orchestration.TableNames.Companion.TMP_TABLE_SUFFIX +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.NO_SUFFIX +import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.TMP_TABLE_SUFFIX import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader import io.github.oshai.kotlinlogging.KotlinLogging @@ -17,13 +17,13 @@ import java.time.Instant private val logger = KotlinLogging.logger {} class TypingDedupingStreamLoader( - override val stream: DestinationStream, - private val initialStatus: TypingDedupingDestinationInitialStatus, - private val tableNames: TableNames, - private val columnNameMapping: ColumnNameMapping, - private val rawTableOperations: TypingDedupingRawTableOperations, - private val finalTableOperations: TypingDedupingFinalTableOperations, - private val disableTypeDedupe: Boolean, + override val stream: DestinationStream, + private val initialStatus: TypingDedupingDestinationInitialStatus, + private val tableNames: TableNames, + private val columnNameMapping: ColumnNameMapping, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : StreamLoader { private val isTruncateSync = when (stream.minimumGenerationId) { @@ -102,7 +102,7 @@ class TypingDedupingStreamLoader( ) // We nuked the temp raw table, so create a new initial raw table status. return Pair( - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } @@ -132,7 +132,7 @@ class TypingDedupingStreamLoader( ) return Pair( // Create a fresh raw table status, since we created a fresh temp stage. - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } @@ -147,7 +147,7 @@ class TypingDedupingStreamLoader( ) return Pair( // Create a fresh raw table status, since we created a fresh temp stage. - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt similarity index 93% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt index a772295f76381..73da32e87318e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingUtil.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt @@ -2,7 +2,7 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import kotlin.math.max diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt similarity index 80% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index b6b9544a0e121..5679f715a200a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -2,13 +2,13 @@ * Copyright (c) 2025 Airbyte, Inc., all rights reserved. */ -package io.airbyte.cdk.load.orchestration.legacy_typing_deduping +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.DestinationHandler -import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer -import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader import java.util.concurrent.Executors @@ -17,13 +17,13 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.runBlocking class TypingDedupingWriter( - private val names: Map>, - private val stateGatherer: - DestinationInitialStatusGatherer, - private val destinationHandler: DestinationHandler, - private val rawTableOperations: TypingDedupingRawTableOperations, - private val finalTableOperations: TypingDedupingFinalTableOperations, - private val disableTypeDedupe: Boolean, + private val names: Map>, + private val stateGatherer: + DestinationInitialStatusGatherer, + private val destinationHandler: DestinationHandler, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : DestinationWriter { private lateinit var initialStatuses: Map diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt index 0a7860410f94c..483de65d2ab83 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt @@ -14,8 +14,8 @@ import com.google.cloud.bigquery.JobStatus import com.google.cloud.bigquery.QueryJobConfiguration import com.google.common.collect.Streams import io.airbyte.cdk.ConfigErrorException -import io.airbyte.cdk.load.orchestration.DestinationHandler -import io.airbyte.cdk.load.orchestration.Sql +import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.Sql import io.airbyte.cdk.util.ConnectorExceptionUtil import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.github.oshai.kotlinlogging.KotlinLogging diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 161fbfb76ef98..84f506d79eaab 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -24,12 +24,12 @@ import io.airbyte.cdk.load.data.TimestampTypeWithTimezone import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone import io.airbyte.cdk.load.data.UnionType import io.airbyte.cdk.load.data.UnknownType -import io.airbyte.cdk.load.orchestration.CDC_DELETED_AT_COLUMN -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.Sql -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingSqlGenerator +import io.airbyte.cdk.load.orchestration.db.CDC_DELETED_AT_COLUMN +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.Sql +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingSqlGenerator import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer import java.time.Instant import java.util.* diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index c07780e4e5b67..4956c7ba1f85c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -19,14 +19,14 @@ import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.Overwrite import io.airbyte.cdk.load.data.ObjectType import io.airbyte.cdk.load.message.Meta -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.DestinationInitialStatusGatherer -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.AlterTableReport -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.FinalTableInitialStatus -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.RawTableInitialStatus -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingDestinationInitialStatus +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.AlterTableReport +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.FinalTableInitialStatus +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.RawTableInitialStatus +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingDestinationInitialStatus import io.airbyte.cdk.util.CollectionUtils.containsAllIgnoreCase import io.airbyte.cdk.util.CollectionUtils.containsIgnoreCase import io.airbyte.cdk.util.CollectionUtils.matchingKey @@ -38,7 +38,7 @@ import java.util.stream.Stream private val logger = KotlinLogging.logger {} class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : - DestinationInitialStatusGatherer { + DestinationInitialStatusGatherer { private fun findExistingTable(finalTableName: TableName): TableDefinition? { val table = bq.getTable(finalTableName.namespace, finalTableName.name) return table?.getDefinition() @@ -50,8 +50,8 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun getInitialRawTableState( - rawTableName: TableName, - suffix: String + rawTableName: TableName, + suffix: String ): RawTableInitialStatus? { bq.getTable(TableId.of(rawTableName.namespace, rawTableName.name + suffix)) // Table doesn't exist. There are no unprocessed records, and no timestamp. @@ -151,9 +151,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun existingSchemaMatchesStreamConfig( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: TableDefinition + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition ): Boolean { val alterTableReport = buildAlterTableReport(stream, columnNameMapping, existingTable) var tableClusteringMatches = false @@ -170,9 +170,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun buildAlterTableReport( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: TableDefinition, + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition, ): AlterTableReport { val pks = getPks(stream, columnNameMapping) @@ -240,9 +240,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : @VisibleForTesting fun clusteringMatches( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: StandardTableDefinition, + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: StandardTableDefinition, ): Boolean { return (existingTable.clustering != null && containsAllIgnoreCase( diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt index 595de616923ed..6674780d58a96 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryNameGenerators.kt @@ -5,11 +5,11 @@ package io.airbyte.integrations.destination.bigquery.typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameGenerator -import io.airbyte.cdk.load.orchestration.FinalTableNameGenerator -import io.airbyte.cdk.load.orchestration.RawTableNameGenerator -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingUtil +import io.airbyte.cdk.load.orchestration.db.ColumnNameGenerator +import io.airbyte.cdk.load.orchestration.db.FinalTableNameGenerator +import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingUtil import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer import java.util.Locale diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt index 98959c2977c71..311add9561cab 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt @@ -9,8 +9,8 @@ import com.google.cloud.bigquery.QueryJobConfiguration import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.TableResult import io.airbyte.cdk.load.message.Meta -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingRawTableOperations +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingRawTableOperations import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter import io.github.oshai.kotlinlogging.KotlinLogging @@ -18,7 +18,7 @@ import io.github.oshai.kotlinlogging.KotlinLogging private val logger = KotlinLogging.logger {} class BigqueryRawTableOperations(private val bigquery: BigQuery) : - TypingDedupingRawTableOperations { + TypingDedupingRawTableOperations { override fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean) { // Prepare staging table. For overwrite, it does drop-create so we can skip explicit create. if (replace) { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index 3e9df98be8b4e..c6b2a397719e0 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -6,10 +6,10 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.TableNames -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingFinalTableOperations -import io.airbyte.cdk.load.orchestration.legacy_typing_deduping.TypingDedupingWriter +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingWriter import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt index fcdf4a187332d..c36303b4966d8 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt @@ -12,9 +12,9 @@ import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.file.gcs.GcsBlob import io.airbyte.cdk.load.file.gcs.GcsClient import io.airbyte.cdk.load.message.StreamKey -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.TableName -import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.TableName +import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.write.db.BulkLoader import io.airbyte.cdk.load.write.db.BulkLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index 6030a658360dd..6ba4d17cf5b67 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -15,8 +15,8 @@ import com.google.cloud.bigquery.WriteChannelConfiguration import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.message.DestinationRecordRaw -import io.airbyte.cdk.load.orchestration.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.TableNames +import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping +import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils From 24d32fa7c3c65eb550b0716dc79c3d8f6de2f2f0 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 08:57:49 -0700 Subject: [PATCH 37/55] formatting --- .../db/DestinationInitialStatusGatherer.kt | 2 +- .../airbyte/cdk/load/orchestration/db/Sql.kt | 4 +++ .../TableNameMapFactory.kt | 10 +++--- .../TypingDedupingDestinationInitialStatus.kt | 6 ++-- .../TypingDedupingFinalTableOperations.kt | 36 +++++++++---------- .../TypingDedupingSqlGenerator.kt | 34 +++++++++--------- .../TypingDedupingStreamLoader.kt | 20 +++++------ .../TypingDedupingWriter.kt | 14 ++++---- ...igqueryDestinationInitialStatusGatherer.kt | 24 ++++++------- .../write/BigqueryRawTableOperations.kt | 2 +- 10 files changed, 78 insertions(+), 74 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt index f75c3489dbc6a..ba68e90fa4902 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt @@ -24,6 +24,6 @@ interface DestinationInitialStatus */ fun interface DestinationInitialStatusGatherer { suspend fun gatherInitialStatus( - streams: Map>, + streams: Map>, ): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt index 25ca51c1d7c1d..6f5caef95cfb5 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/Sql.kt @@ -1,3 +1,7 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + package io.airbyte.cdk.load.orchestration.db import java.util.function.Consumer diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt index 8f309c62ec792..07aa08994e704 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt @@ -16,10 +16,10 @@ import javax.inject.Singleton @Factory class TypingDedupingTableNameMapFactory( - private val catalog: DestinationCatalog, - private val rawTableNameGenerator: RawTableNameGenerator, - private val finalTableNameGenerator: FinalTableNameGenerator, - private val finalTableColumnNameGenerator: ColumnNameGenerator, + private val catalog: DestinationCatalog, + private val rawTableNameGenerator: RawTableNameGenerator, + private val finalTableNameGenerator: FinalTableNameGenerator, + private val finalTableColumnNameGenerator: ColumnNameGenerator, ) { @Singleton fun get(): Map> { @@ -43,7 +43,7 @@ class TypingDedupingTableNameMapFactory( @Factory class TypingDedupingTableNameMapByDescriptorFactory( - private val map: Map>, + private val map: Map>, ) { @Singleton fun get(): Map> { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt index 971bc06e51f42..da942b77313d6 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt @@ -8,10 +8,10 @@ import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatus import java.time.Instant data class TypingDedupingDestinationInitialStatus( - /** Initial status of the final table, or null if the table doesn't exist yet. */ + /** Initial status of the final table, or null if the table doesn't exist yet. */ val finalTableStatus: FinalTableInitialStatus?, - val rawTableStatus: RawTableInitialStatus?, - val tempRawTableStatus: RawTableInitialStatus?, + val rawTableStatus: RawTableInitialStatus?, + val tempRawTableStatus: RawTableInitialStatus?, ) : DestinationInitialStatus data class FinalTableInitialStatus( diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 3e78ea59bf094..4c336e32a5e9f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -16,15 +16,15 @@ import java.time.Instant private val logger = KotlinLogging.logger {} class TypingDedupingFinalTableOperations( - private val sqlGenerator: TypingDedupingSqlGenerator, - private val destinationHandler: DestinationHandler, + private val sqlGenerator: TypingDedupingSqlGenerator, + private val destinationHandler: DestinationHandler, ) { fun createFinalTable( - stream: DestinationStream, - finalTableName: TableName, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - replace: Boolean + stream: DestinationStream, + finalTableName: TableName, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + replace: Boolean ) { logger.info { "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()}" @@ -42,9 +42,9 @@ class TypingDedupingFinalTableOperations( /** Reset the final table using a temp table or ALTER existing table's columns. */ fun softResetFinalTable( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, ) { logger.info { "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" @@ -73,9 +73,9 @@ class TypingDedupingFinalTableOperations( * specific, INSERT INTO..SELECT * and DROP TABLE OR CREATE OR REPLACE ... SELECT *, DROP TABLE */ fun overwriteFinalTable( - stream: DestinationStream, - finalTableName: TableName, - finalTableSuffix: String, + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String, ) { logger.info { "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" @@ -90,11 +90,11 @@ class TypingDedupingFinalTableOperations( } fun typeAndDedupe( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, - maxProcessedTimestamp: Instant?, - finalTableSuffix: String + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + maxProcessedTimestamp: Instant?, + finalTableSuffix: String ) { try { logger.info { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt index 43a957a0cc7d2..c899800286453 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingSqlGenerator.kt @@ -26,11 +26,11 @@ interface TypingDedupingSqlGenerator { * this to true. */ fun createFinalTable( - stream: DestinationStream, - tableName: TableName, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - replace: Boolean + stream: DestinationStream, + tableName: TableName, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + replace: Boolean ): Sql /** @@ -69,12 +69,12 @@ interface TypingDedupingSqlGenerator { * which handles casting exceptions. */ fun updateFinalTable( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, - finalTableSuffix: String, - maxProcessedTimestamp: Instant?, - useExpensiveSaferCasting: Boolean, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, + finalTableSuffix: String, + maxProcessedTimestamp: Instant?, + useExpensiveSaferCasting: Boolean, ): Sql /** @@ -84,18 +84,18 @@ interface TypingDedupingSqlGenerator { * non-empty. Callers are responsible for verifying those are true. */ fun overwriteFinalTable( - stream: DestinationStream, - finalTableName: TableName, - finalTableSuffix: String, + stream: DestinationStream, + finalTableName: TableName, + finalTableSuffix: String, ): Sql fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql /** Typically we need to create a soft reset temporary table and clear loaded at values */ fun prepareTablesForSoftReset( - stream: DestinationStream, - tableNames: TableNames, - columnNameMapping: ColumnNameMapping, + stream: DestinationStream, + tableNames: TableNames, + columnNameMapping: ColumnNameMapping, ): Sql { val createTempTable = createFinalTable( diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 007e48035e77e..3b139a78f73f6 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -17,13 +17,13 @@ import java.time.Instant private val logger = KotlinLogging.logger {} class TypingDedupingStreamLoader( - override val stream: DestinationStream, - private val initialStatus: TypingDedupingDestinationInitialStatus, - private val tableNames: TableNames, - private val columnNameMapping: ColumnNameMapping, - private val rawTableOperations: TypingDedupingRawTableOperations, - private val finalTableOperations: TypingDedupingFinalTableOperations, - private val disableTypeDedupe: Boolean, + override val stream: DestinationStream, + private val initialStatus: TypingDedupingDestinationInitialStatus, + private val tableNames: TableNames, + private val columnNameMapping: ColumnNameMapping, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : StreamLoader { private val isTruncateSync = when (stream.minimumGenerationId) { @@ -102,7 +102,7 @@ class TypingDedupingStreamLoader( ) // We nuked the temp raw table, so create a new initial raw table status. return Pair( - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } @@ -132,7 +132,7 @@ class TypingDedupingStreamLoader( ) return Pair( // Create a fresh raw table status, since we created a fresh temp stage. - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } @@ -147,7 +147,7 @@ class TypingDedupingStreamLoader( ) return Pair( // Create a fresh raw table status, since we created a fresh temp stage. - RawTableInitialStatus.emptyTableStatus, + RawTableInitialStatus.emptyTableStatus, TMP_TABLE_SUFFIX, ) } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index 5679f715a200a..c799aaab30cd2 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -17,13 +17,13 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.runBlocking class TypingDedupingWriter( - private val names: Map>, - private val stateGatherer: - DestinationInitialStatusGatherer, - private val destinationHandler: DestinationHandler, - private val rawTableOperations: TypingDedupingRawTableOperations, - private val finalTableOperations: TypingDedupingFinalTableOperations, - private val disableTypeDedupe: Boolean, + private val names: Map>, + private val stateGatherer: + DestinationInitialStatusGatherer, + private val destinationHandler: DestinationHandler, + private val rawTableOperations: TypingDedupingRawTableOperations, + private val finalTableOperations: TypingDedupingFinalTableOperations, + private val disableTypeDedupe: Boolean, ) : DestinationWriter { private lateinit var initialStatuses: Map diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index 4956c7ba1f85c..38a9f0ddd3c04 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -38,7 +38,7 @@ import java.util.stream.Stream private val logger = KotlinLogging.logger {} class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : - DestinationInitialStatusGatherer { + DestinationInitialStatusGatherer { private fun findExistingTable(finalTableName: TableName): TableDefinition? { val table = bq.getTable(finalTableName.namespace, finalTableName.name) return table?.getDefinition() @@ -50,8 +50,8 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun getInitialRawTableState( - rawTableName: TableName, - suffix: String + rawTableName: TableName, + suffix: String ): RawTableInitialStatus? { bq.getTable(TableId.of(rawTableName.namespace, rawTableName.name + suffix)) // Table doesn't exist. There are no unprocessed records, and no timestamp. @@ -151,9 +151,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun existingSchemaMatchesStreamConfig( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: TableDefinition + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition ): Boolean { val alterTableReport = buildAlterTableReport(stream, columnNameMapping, existingTable) var tableClusteringMatches = false @@ -170,9 +170,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } private fun buildAlterTableReport( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: TableDefinition, + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: TableDefinition, ): AlterTableReport { val pks = getPks(stream, columnNameMapping) @@ -240,9 +240,9 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : @VisibleForTesting fun clusteringMatches( - stream: DestinationStream, - columnNameMapping: ColumnNameMapping, - existingTable: StandardTableDefinition, + stream: DestinationStream, + columnNameMapping: ColumnNameMapping, + existingTable: StandardTableDefinition, ): Boolean { return (existingTable.clustering != null && containsAllIgnoreCase( diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt index 311add9561cab..0b92f36990281 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryRawTableOperations.kt @@ -18,7 +18,7 @@ import io.github.oshai.kotlinlogging.KotlinLogging private val logger = KotlinLogging.logger {} class BigqueryRawTableOperations(private val bigquery: BigQuery) : - TypingDedupingRawTableOperations { + TypingDedupingRawTableOperations { override fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean) { // Prepare staging table. For overwrite, it does drop-create so we can skip explicit create. if (replace) { From bafeb80006c7c0d94b78381afc6d8f1dfe7bd6fa Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 08:58:52 -0700 Subject: [PATCH 38/55] rename destination -> databasehandler --- ...{DestinationHandler.kt => DatabaseHandler.kt} | 2 +- .../DirectLoadTableOperations.kt | 6 +++--- .../TypingDedupingFinalTableOperations.kt | 16 ++++++++-------- .../TypingDedupingWriter.kt | 6 +++--- .../LegacyBigQuerySqlGenerator.kt | 5 ++--- ...tionHandler.kt => BigQueryDatabaseHandler.kt} | 6 +++--- .../typing_deduping/BigQuerySqlGenerator.kt | 5 ++--- .../destination/bigquery/write/BigqueryWriter.kt | 4 ++-- .../BigqueryDestinationHandlerTest.kt | 7 +++---- 9 files changed, 27 insertions(+), 30 deletions(-) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/{DestinationHandler.kt => DatabaseHandler.kt} (94%) rename airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/{BigQueryDestinationHandler.kt => BigQueryDatabaseHandler.kt} (96%) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseHandler.kt similarity index 94% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseHandler.kt index a9d03382b49f6..46c202e35a678 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationHandler.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseHandler.kt @@ -4,7 +4,7 @@ package io.airbyte.cdk.load.orchestration.db -interface DestinationHandler { +interface DatabaseHandler { fun execute(sql: Sql) /** diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt index c948995d73c79..61d55050f0aab 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/direct_load_table/DirectLoadTableOperations.kt @@ -5,12 +5,12 @@ package io.airbyte.cdk.load.orchestration.db.direct_load_table import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.TableName class DirectLoadTableOperations( private val sqlGenerator: DirectLoadSqlGenerator, - private val destinationHandler: DestinationHandler, + private val databaseHandler: DatabaseHandler, ) { fun createTable( stream: DestinationStream, @@ -18,7 +18,7 @@ class DirectLoadTableOperations( suffix: String, replace: Boolean ) { - destinationHandler.execute(TODO()) + databaseHandler.execute(TODO()) } fun alterTable( diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 4c336e32a5e9f..2b68731e80414 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -6,7 +6,7 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.TableName import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.SOFT_RESET_SUFFIX @@ -17,7 +17,7 @@ private val logger = KotlinLogging.logger {} class TypingDedupingFinalTableOperations( private val sqlGenerator: TypingDedupingSqlGenerator, - private val destinationHandler: DestinationHandler, + private val databaseHandler: DatabaseHandler, ) { fun createFinalTable( stream: DestinationStream, @@ -29,7 +29,7 @@ class TypingDedupingFinalTableOperations( logger.info { "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()}" } - destinationHandler.execute( + databaseHandler.execute( sqlGenerator.createFinalTable( stream, finalTableName, @@ -49,7 +49,7 @@ class TypingDedupingFinalTableOperations( logger.info { "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" } - destinationHandler.execute( + databaseHandler.execute( sqlGenerator.prepareTablesForSoftReset(stream, tableNames, columnNameMapping) ) typeAndDedupe( @@ -59,7 +59,7 @@ class TypingDedupingFinalTableOperations( maxProcessedTimestamp = null, finalTableSuffix = SOFT_RESET_SUFFIX, ) - destinationHandler.execute( + databaseHandler.execute( sqlGenerator.overwriteFinalTable( stream, tableNames.finalTableName!!, @@ -80,7 +80,7 @@ class TypingDedupingFinalTableOperations( logger.info { "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" } - destinationHandler.execute( + databaseHandler.execute( sqlGenerator.overwriteFinalTable( stream, finalTableName, @@ -109,7 +109,7 @@ class TypingDedupingFinalTableOperations( maxProcessedTimestamp = maxProcessedTimestamp, useExpensiveSaferCasting = false, ) - destinationHandler.execute(unsafeSql) + databaseHandler.execute(unsafeSql) } catch (e: Exception) { if (sqlGenerator.supportsExpensiveSaferCasting) { logger.info(e) { @@ -124,7 +124,7 @@ class TypingDedupingFinalTableOperations( maxProcessedTimestamp = maxProcessedTimestamp, useExpensiveSaferCasting = true, ) - destinationHandler.execute(saferSql) + databaseHandler.execute(saferSql) } else { logger.info(e) { "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, not retrying" diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index c799aaab30cd2..4753152f69c4f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -6,7 +6,7 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.write.DestinationWriter @@ -20,7 +20,7 @@ class TypingDedupingWriter( private val names: Map>, private val stateGatherer: DestinationInitialStatusGatherer, - private val destinationHandler: DestinationHandler, + private val databaseHandler: DatabaseHandler, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, @@ -33,7 +33,7 @@ class TypingDedupingWriter( val namespaces = names.values.map { (tableNames, _) -> tableNames.rawTableName!!.namespace } + names.values.map { (tableNames, _) -> tableNames.finalTableName!!.namespace } - destinationHandler.createNamespaces(namespaces.toSet()) + databaseHandler.createNamespaces(namespaces.toSet()) val initialInitialStatuses: Map = diff --git a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt index c6279d41091ea..23b8ad49ebf43 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/integrationTestLegacy/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/LegacyBigQuerySqlGenerator.kt @@ -23,9 +23,8 @@ import org.slf4j.LoggerFactory class LegacyBigQuerySqlGenerator /** * @param projectId - * @param datasetLocation This is technically redundant with [BigQueryDestinationHandler] setting - * the query execution location, but let's be explicit since this is typically a compliance - * requirement. + * @param datasetLocation This is technically redundant with [BigQueryDatabaseHandler] setting the + * query execution location, but let's be explicit since this is typically a compliance requirement. */ (private val projectId: String?, private val datasetLocation: String?) : SqlGenerator { private val CDC_DELETED_AT_COLUMN = buildColumnId("_ab_cdc_deleted_at") diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt similarity index 96% rename from airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt rename to airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt index 483de65d2ab83..5b221a0b4d82c 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDestinationHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt @@ -14,7 +14,7 @@ import com.google.cloud.bigquery.JobStatus import com.google.cloud.bigquery.QueryJobConfiguration import com.google.common.collect.Streams import io.airbyte.cdk.ConfigErrorException -import io.airbyte.cdk.load.orchestration.db.DestinationHandler +import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.Sql import io.airbyte.cdk.util.ConnectorExceptionUtil import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -27,8 +27,8 @@ import kotlinx.coroutines.launch private val logger = KotlinLogging.logger {} -class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLocation: String) : - DestinationHandler { +class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocation: String) : + DatabaseHandler { @Throws(InterruptedException::class) override fun execute(sql: Sql) { val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION") diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index 84f506d79eaab..c1af8b14f0bfe 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -38,9 +38,8 @@ import org.apache.commons.lang3.StringUtils /** * @param projectId - * @param datasetLocation This is technically redundant with [BigQueryDestinationHandler] setting - * the query execution location, but let's be explicit since this is typically a compliance - * requirement. + * @param datasetLocation This is technically redundant with [BigQueryDatabaseHandler] setting the + * query execution location, but let's be explicit since this is typically a compliance requirement. */ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLocation: String?) : TypingDedupingSqlGenerator { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index c6b2a397719e0..8f717b2ef83b5 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -11,7 +11,7 @@ import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingWriter import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDestinationInitialStatusGatherer import io.micronaut.context.annotation.Factory @@ -25,7 +25,7 @@ class BigqueryWriterFactory( ) { @Singleton fun make(): TypingDedupingWriter { - val destinationHandler = BigQueryDestinationHandler(bigquery, config.datasetLocation.region) + val destinationHandler = BigQueryDatabaseHandler(bigquery, config.datasetLocation.region) return TypingDedupingWriter( names, BigqueryDestinationInitialStatusGatherer(bigquery), diff --git a/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationHandlerTest.kt b/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationHandlerTest.kt index b43f950fbb3e6..85d3be88048e5 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationHandlerTest.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/test/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationHandlerTest.kt @@ -10,16 +10,15 @@ import com.google.cloud.bigquery.TimePartitioning import com.google.common.collect.ImmutableList import io.airbyte.integrations.base.destination.typing_deduping.AirbyteProtocolType import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType -import io.airbyte.integrations.base.destination.typing_deduping.Array import io.airbyte.integrations.base.destination.typing_deduping.ColumnId import io.airbyte.integrations.base.destination.typing_deduping.ImportType import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig import io.airbyte.integrations.base.destination.typing_deduping.Struct import io.airbyte.integrations.base.destination.typing_deduping.Union import io.airbyte.integrations.base.destination.typing_deduping.UnsupportedOneOf -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.clusteringMatches -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.partitioningMatches -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.schemaContainAllFinalTableV2AirbyteColumns +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler.Companion.clusteringMatches +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler.Companion.partitioningMatches +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler.Companion.schemaContainAllFinalTableV2AirbyteColumns import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.toDialectType import java.util.Optional import java.util.stream.Collectors From 590a9042b887dad23fad858f8f83b00ffc95d91b Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 09:05:40 -0700 Subject: [PATCH 39/55] give this map a name --- .../db/DestinationInitialStatusGatherer.kt | 5 +- .../TableNameMapFactory.kt | 51 ++++++++++++------- .../TypingDedupingWriter.kt | 4 +- ...igqueryDestinationInitialStatusGatherer.kt | 3 +- .../bigquery/write/BigqueryWriter.kt | 6 +-- .../write/bulk_loader/BigQueryBulkLoader.kt | 8 ++- .../BigqueryBatchStandardInsertLoader.kt | 7 ++- 7 files changed, 45 insertions(+), 39 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt index ba68e90fa4902..46272a3ec5e4a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt @@ -5,6 +5,7 @@ package io.airbyte.cdk.load.orchestration.db import io.airbyte.cdk.load.command.DestinationStream +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog interface DestinationInitialStatus @@ -23,7 +24,5 @@ interface DestinationInitialStatus * ``` */ fun interface DestinationInitialStatusGatherer { - suspend fun gatherInitialStatus( - streams: Map>, - ): Map + suspend fun gatherInitialStatus(streams: TableCatalog): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt index 07aa08994e704..8f86d5aed4e76 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt @@ -14,39 +14,52 @@ import io.airbyte.cdk.load.orchestration.db.TableNames import io.micronaut.context.annotation.Factory import javax.inject.Singleton +data class TableNameInfo(val tableNames: TableNames, val columnNameMapping: ColumnNameMapping) + +@JvmInline +value class TableCatalog(private val catalog: Map) : + Map by catalog + +@JvmInline +value class TableCatalogByDescriptor( + private val catalog: Map +) : Map by catalog + @Factory -class TypingDedupingTableNameMapFactory( +class TableCatalogFactory( private val catalog: DestinationCatalog, private val rawTableNameGenerator: RawTableNameGenerator, private val finalTableNameGenerator: FinalTableNameGenerator, private val finalTableColumnNameGenerator: ColumnNameGenerator, ) { @Singleton - fun get(): Map> { + fun get(): TableCatalog { // TODO handle collisions in table names - return catalog.streams.associateWith { stream -> - Pair( - TableNames( - rawTableName = rawTableNameGenerator.getTableName(stream.descriptor), - finalTableName = finalTableNameGenerator.getTableName(stream.descriptor), - ), - ColumnNameMapping( - // TODO handle collisions in column names - stream.schema.asColumns().mapValues { (columnName, _) -> - finalTableColumnNameGenerator.getColumnName(columnName).displayName - } + return TableCatalog( + catalog.streams.associateWith { stream -> + TableNameInfo( + TableNames( + rawTableName = rawTableNameGenerator.getTableName(stream.descriptor), + finalTableName = finalTableNameGenerator.getTableName(stream.descriptor), + ), + ColumnNameMapping( + // TODO handle collisions in column names + stream.schema.asColumns().mapValues { (columnName, _) -> + finalTableColumnNameGenerator.getColumnName(columnName).displayName + } + ) ) - ) - } + } + ) } } @Factory -class TypingDedupingTableNameMapByDescriptorFactory( - private val map: Map>, +class TableCatalogByDescriptorFactory( + private val map: TableCatalog, ) { @Singleton - fun get(): Map> { - return map.mapKeys { (k, _) -> k.descriptor } + fun get(): TableCatalogByDescriptor { + return TableCatalogByDescriptor(map.mapKeys { (k, _) -> k.descriptor }) } } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index 4753152f69c4f..0d81792fe37e7 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -5,10 +5,8 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer -import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader import java.util.concurrent.Executors @@ -17,7 +15,7 @@ import kotlinx.coroutines.launch import kotlinx.coroutines.runBlocking class TypingDedupingWriter( - private val names: Map>, + private val names: TableCatalog, private val stateGatherer: DestinationInitialStatusGatherer, private val databaseHandler: DatabaseHandler, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt index 38a9f0ddd3c04..bb8d9f134c599 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt @@ -26,6 +26,7 @@ import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.AlterTableReport import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.FinalTableInitialStatus import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.RawTableInitialStatus +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingDestinationInitialStatus import io.airbyte.cdk.util.CollectionUtils.containsAllIgnoreCase import io.airbyte.cdk.util.CollectionUtils.containsIgnoreCase @@ -114,7 +115,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : } override suspend fun gatherInitialStatus( - streams: Map> + streams: TableCatalog, ): Map { return streams.mapValues { (stream, names) -> val (tableNames, columnNameMapping) = names diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index 8f717b2ef83b5..f02713c0e6e82 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -5,9 +5,7 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery -import io.airbyte.cdk.load.command.DestinationStream -import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingWriter import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration @@ -21,7 +19,7 @@ import jakarta.inject.Singleton class BigqueryWriterFactory( private val bigquery: BigQuery, private val config: BigqueryConfiguration, - private val names: Map>, + private val names: TableCatalog, ) { @Singleton fun make(): TypingDedupingWriter { diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt index c36303b4966d8..1f07757e83b19 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt @@ -8,13 +8,11 @@ import com.google.cloud.bigquery.* import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.LoadJobConfiguration -import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.file.gcs.GcsBlob import io.airbyte.cdk.load.file.gcs.GcsClient import io.airbyte.cdk.load.message.StreamKey -import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping import io.airbyte.cdk.load.orchestration.db.TableName -import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor import io.airbyte.cdk.load.write.db.BulkLoader import io.airbyte.cdk.load.write.db.BulkLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -85,7 +83,7 @@ class BigqueryConfiguredForBulkLoad : Condition { @Singleton @Requires(condition = BigqueryConfiguredForBulkLoad::class) class BigQueryBulkLoaderFactory( - private val names: Map>, + private val names: TableCatalogByDescriptor, private val storageClient: GcsClient, private val bigQueryClient: BigQuery, private val bigQueryConfiguration: BigqueryConfiguration @@ -103,7 +101,7 @@ class BigQueryBulkLoaderFactory( storageClient, bigQueryClient, bigQueryConfiguration, - names[key.stream]!!.first.rawTableName!!, + names[key.stream]!!.tableNames.rawTableName!!, ) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index 6ba4d17cf5b67..b495879ae0f44 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -15,8 +15,7 @@ import com.google.cloud.bigquery.WriteChannelConfiguration import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.message.DestinationRecordRaw -import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.db.TableNames +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -68,13 +67,13 @@ class BigqueryConfiguredForBatchStandardInserts : Condition { class BigqueryBatchStandardInsertsLoaderFactory( private val bigquery: BigQuery, private val config: BigqueryConfiguration, - private val names: Map>, + private val names: TableCatalogByDescriptor, ) : DirectLoaderFactory { override fun create( streamDescriptor: DestinationStream.Descriptor, part: Int, ): BigqueryBatchStandardInsertsLoader { - val tableName = names[streamDescriptor]!!.first.rawTableName!! + val tableName = names[streamDescriptor]!!.tableNames.rawTableName!! val writeChannelConfiguration = WriteChannelConfiguration.newBuilder(TableId.of(tableName.namespace, tableName.name)) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) From ef252ef6055b613d210132803c4bee2ccdf61917 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 09:07:38 -0700 Subject: [PATCH 40/55] airbytetype is abstract class now --- .../io/airbyte/cdk/load/data/AirbyteType.kt | 42 +++++++++---------- 1 file changed, 20 insertions(+), 22 deletions(-) diff --git a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt index 53280eabd00d1..9ec50995750f3 100644 --- a/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt +++ b/airbyte-cdk/bulk/core/load/src/main/kotlin/io/airbyte/cdk/load/data/AirbyteType.kt @@ -10,48 +10,46 @@ import io.github.oshai.kotlinlogging.KotlinLogging private val logger = KotlinLogging.logger {} -sealed interface AirbyteType { +sealed class AirbyteType { /** * Utility method for database/warehouse destinations, which assume that the top-level schema is * an object. */ - fun asColumns(): LinkedHashMap { + open fun asColumns(): LinkedHashMap { return linkedMapOf() } - val isObject: Boolean - get() = false - val isArray: Boolean - get() = false + open val isObject: Boolean = false + open val isArray: Boolean = false } -data object StringType : AirbyteType +data object StringType : AirbyteType() -data object BooleanType : AirbyteType +data object BooleanType : AirbyteType() -data object IntegerType : AirbyteType +data object IntegerType : AirbyteType() -data object NumberType : AirbyteType +data object NumberType : AirbyteType() -data object DateType : AirbyteType +data object DateType : AirbyteType() -data object TimestampTypeWithTimezone : AirbyteType +data object TimestampTypeWithTimezone : AirbyteType() -data object TimestampTypeWithoutTimezone : AirbyteType +data object TimestampTypeWithoutTimezone : AirbyteType() -data object TimeTypeWithTimezone : AirbyteType +data object TimeTypeWithTimezone : AirbyteType() -data object TimeTypeWithoutTimezone : AirbyteType +data object TimeTypeWithoutTimezone : AirbyteType() -data class ArrayType(val items: FieldType) : AirbyteType { +data class ArrayType(val items: FieldType) : AirbyteType() { override val isArray = true } -data object ArrayTypeWithoutSchema : AirbyteType { +data object ArrayTypeWithoutSchema : AirbyteType() { override val isArray = true } -data class ObjectType(val properties: LinkedHashMap) : AirbyteType { +data class ObjectType(val properties: LinkedHashMap) : AirbyteType() { override fun asColumns(): LinkedHashMap { return properties } @@ -59,18 +57,18 @@ data class ObjectType(val properties: LinkedHashMap) : Airbyt override val isObject = true } -data object ObjectTypeWithEmptySchema : AirbyteType { +data object ObjectTypeWithEmptySchema : AirbyteType() { override val isObject = true } -data object ObjectTypeWithoutSchema : AirbyteType { +data object ObjectTypeWithoutSchema : AirbyteType() { override val isObject = true } data class UnionType( val options: Set, val isLegacyUnion: Boolean, -) : AirbyteType { +) : AirbyteType() { /** * This is a hack to handle weird schemas like {type: [object, string]}. If a stream's top-level * schema looks like this, we still want to be able to extract the object properties (i.e. treat @@ -144,6 +142,6 @@ data class UnionType( } } -data class UnknownType(val schema: JsonNode) : AirbyteType +data class UnknownType(val schema: JsonNode) : AirbyteType() data class FieldType(val type: AirbyteType, val nullable: Boolean) From b4268cf027917f05ef941b333ccb19957d01b182 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 09:54:23 -0700 Subject: [PATCH 41/55] rename method --- .../airbyte/cdk/load/orchestration/db/DestinationNames.kt | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt index 9cb6f2067473a..02409eb3786b1 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt @@ -20,9 +20,9 @@ data class TableNames( } } - fun conflictsWith(other: TableNames) = - this.rawTableName.conflictsWith(other.rawTableName) || - this.finalTableName.conflictsWith(other.finalTableName) + fun hasNamingConflictWith(other: TableNames) = + this.rawTableName.hasNamingConflictWith(other.rawTableName) || + this.finalTableName.hasNamingConflictWith(other.finalTableName) fun prettyPrint() = "Raw table: ${rawTableName?.prettyPrint()}; Final table: ${finalTableName?.prettyPrint()}" @@ -40,7 +40,7 @@ data class TableName(val namespace: String, val name: String) { "$quote$namespace$quote.$quote$name$suffix$quote" } -fun TableName?.conflictsWith(other: TableName?): Boolean { +fun TableName?.hasNamingConflictWith(other: TableName?): Boolean { if (this == null || other == null) { return false } From 97dce7abf718ba1908ab60dd8bbdd4fb5835eb84 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 09:55:28 -0700 Subject: [PATCH 42/55] rename method --- .../load/orchestration/db/DestinationNames.kt | 6 +++--- .../TypingDedupingFinalTableOperations.kt | 12 ++++++------ .../TypingDedupingStreamLoader.kt | 4 ++-- .../typing_deduping/BigQuerySqlGenerator.kt | 18 +++++++++--------- 4 files changed, 20 insertions(+), 20 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt index 02409eb3786b1..bb6423a52586f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationNames.kt @@ -24,8 +24,8 @@ data class TableNames( this.rawTableName.hasNamingConflictWith(other.rawTableName) || this.finalTableName.hasNamingConflictWith(other.finalTableName) - fun prettyPrint() = - "Raw table: ${rawTableName?.prettyPrint()}; Final table: ${finalTableName?.prettyPrint()}" + fun toPrettyString() = + "Raw table: ${rawTableName?.toPrettyString()}; Final table: ${finalTableName?.toPrettyString()}" companion object { const val NO_SUFFIX = "" @@ -36,7 +36,7 @@ data class TableNames( } data class TableName(val namespace: String, val name: String) { - fun prettyPrint(quote: String = "", suffix: String = "") = + fun toPrettyString(quote: String = "", suffix: String = "") = "$quote$namespace$quote.$quote$name$suffix$quote" } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt index 2b68731e80414..edf0b28a1b01e 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingFinalTableOperations.kt @@ -27,7 +27,7 @@ class TypingDedupingFinalTableOperations( replace: Boolean ) { logger.info { - "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()}" + "Creating final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.toPrettyString()}" } databaseHandler.execute( sqlGenerator.createFinalTable( @@ -47,7 +47,7 @@ class TypingDedupingFinalTableOperations( columnNameMapping: ColumnNameMapping, ) { logger.info { - "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()}" + "Executing soft reset for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.toPrettyString()}" } databaseHandler.execute( sqlGenerator.prepareTablesForSoftReset(stream, tableNames, columnNameMapping) @@ -78,7 +78,7 @@ class TypingDedupingFinalTableOperations( finalTableSuffix: String, ) { logger.info { - "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.prettyPrint()} using temp table with suffix $finalTableSuffix" + "Overwriting final table for stream ${stream.descriptor.toPrettyString()} with name ${finalTableName.toPrettyString()} using temp table with suffix $finalTableSuffix" } databaseHandler.execute( sqlGenerator.overwriteFinalTable( @@ -98,7 +98,7 @@ class TypingDedupingFinalTableOperations( ) { try { logger.info { - "Attempting typing and deduping for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix" + "Attempting typing and deduping for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.toPrettyString()} with suffix $finalTableSuffix" } val unsafeSql = sqlGenerator.updateFinalTable( @@ -113,7 +113,7 @@ class TypingDedupingFinalTableOperations( } catch (e: Exception) { if (sqlGenerator.supportsExpensiveSaferCasting) { logger.info(e) { - "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, re-attempting with error handling" + "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.toPrettyString()} with suffix $finalTableSuffix, re-attempting with error handling" } val saferSql = sqlGenerator.updateFinalTable( @@ -127,7 +127,7 @@ class TypingDedupingFinalTableOperations( databaseHandler.execute(saferSql) } else { logger.info(e) { - "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.prettyPrint()} with suffix $finalTableSuffix, not retrying" + "Encountered Exception on unsafe SQL for stream ${stream.descriptor.toPrettyString()} on tables ${tableNames.toPrettyString()} with suffix $finalTableSuffix, not retrying" } throw e } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 3b139a78f73f6..07047dec1e5df 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -211,7 +211,7 @@ class TypingDedupingStreamLoader( // No special handling if final table doesn't exist, just create and return if (initialStatus.finalTableStatus == null) { logger.info { - "Final table does not exist for stream ${stream.descriptor.toPrettyString()}, creating ${tableNames.finalTableName!!.prettyPrint()}." + "Final table does not exist for stream ${stream.descriptor.toPrettyString()}, creating ${tableNames.finalTableName!!.toPrettyString()}." } finalTableOperations.createFinalTable( stream, @@ -323,7 +323,7 @@ class TypingDedupingStreamLoader( if (disableTypeDedupe) { logger.info { - "Typing and deduping disabled, skipping final table finalization. Raw records can be found at ${tableNames.rawTableName!!.prettyPrint()}" + "Typing and deduping disabled, skipping final table finalization. Raw records can be found at ${tableNames.rawTableName!!.toPrettyString()}" } return } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt index c1af8b14f0bfe..4eb21bf943023 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQuerySqlGenerator.kt @@ -134,7 +134,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo .map { c: String? -> StringUtils.wrap(c, QUOTE) } .collect(Collectors.joining(", ")) val forceCreateTable = if (replace) "OR REPLACE" else "" - val finalTableId = tableName.prettyPrint(QUOTE, finalTableSuffix) + val finalTableId = tableName.toPrettyString(QUOTE, finalTableSuffix) return Sql.of( """ CREATE $forceCreateTable TABLE `$projectId`.$finalTableId ( @@ -196,12 +196,12 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo finalTableName: TableName, suffix: String, ): Sql { - val tableId = finalTableName.prettyPrint(QUOTE, suffix) + val tableId = finalTableName.toPrettyString(QUOTE, suffix) return Sql.of("""DROP TABLE IF EXISTS `$projectId`.$tableId;""") } override fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql { - val rawTableId = rawTableName.prettyPrint(QUOTE) + val rawTableId = rawTableName.toPrettyString(QUOTE) return Sql.of( """UPDATE `$projectId`.$rawTableId SET _airbyte_loaded_at = NULL WHERE 1=1;""" ) @@ -266,7 +266,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo forceSafeCasting, minRawTimestamp ) - val finalTableId = tableNames.finalTableName!!.prettyPrint(QUOTE, finalSuffix) + val finalTableId = tableNames.finalTableName!!.toPrettyString(QUOTE, finalSuffix) return """ INSERT INTO `$projectId`.$finalTableId @@ -357,7 +357,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo val column = columnNameMapping[fieldName]!! "`$column` = new_record.`$column`," } - val finalTableId = tableNames.finalTableName!!.prettyPrint(QUOTE, finalSuffix) + val finalTableId = tableNames.finalTableName!!.toPrettyString(QUOTE, finalSuffix) return """ MERGE `$projectId`.$finalTableId target_table @@ -448,7 +448,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo } val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) - val rawTableId = tableNames.rawTableName!!.prettyPrint(QUOTE) + val rawTableId = tableNames.rawTableName!!.toPrettyString(QUOTE) if (stream.importType is Dedupe) { val importType = stream.importType as Dedupe // When deduping, we need to dedup the raw records. Note the row_number() invocation in @@ -570,7 +570,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo @VisibleForTesting fun commitRawTable(rawTableName: TableName, minRawTimestamp: Instant?): String { - val rawTableId = rawTableName.prettyPrint(QUOTE) + val rawTableId = rawTableName.toPrettyString(QUOTE) val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp) return """ UPDATE `$projectId`.$rawTableId @@ -586,8 +586,8 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo finalTableName: TableName, finalTableSuffix: String, ): Sql { - val finalTableId = finalTableName.prettyPrint(QUOTE) - val tempFinalTableId = finalTableName.prettyPrint(QUOTE, finalTableSuffix) + val finalTableId = finalTableName.toPrettyString(QUOTE) + val tempFinalTableId = finalTableName.toPrettyString(QUOTE, finalTableSuffix) return Sql.separately( "DROP TABLE IF EXISTS `$projectId`.$finalTableId;", "ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO `${finalTableName.name}`;" From 02cc7cc1e5fa16f993b23b0bb68f4840744ca8fa Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 09:57:41 -0700 Subject: [PATCH 43/55] rename class --- ...sGatherer.kt => DatabaseInitialStatusGatherer.kt} | 4 ++-- ...tus.kt => TypingDedupingDatabaseInitialStatus.kt} | 6 +++--- .../TypingDedupingStreamLoader.kt | 2 +- .../legacy_typing_deduping/TypingDedupingWriter.kt | 9 ++++----- ...r.kt => BigqueryDatabaseInitialStatusGatherer.kt} | 12 ++++++------ .../destination/bigquery/write/BigqueryWriter.kt | 4 ++-- 6 files changed, 18 insertions(+), 19 deletions(-) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/{DestinationInitialStatusGatherer.kt => DatabaseInitialStatusGatherer.kt} (86%) rename airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/{TypingDedupingDestinationInitialStatus.kt => TypingDedupingDatabaseInitialStatus.kt} (92%) rename airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/{BigqueryDestinationInitialStatusGatherer.kt => BigqueryDatabaseInitialStatusGatherer.kt} (96%) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseInitialStatusGatherer.kt similarity index 86% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseInitialStatusGatherer.kt index 46272a3ec5e4a..727445fd94bea 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DestinationInitialStatusGatherer.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/DatabaseInitialStatusGatherer.kt @@ -7,7 +7,7 @@ package io.airbyte.cdk.load.orchestration.db import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog -interface DestinationInitialStatus +interface DatabaseInitialStatus /** * Some destinations can efficiently fetch multiple tables' information in a single query, so this @@ -23,6 +23,6 @@ interface DestinationInitialStatus * } * ``` */ -fun interface DestinationInitialStatusGatherer { +fun interface DatabaseInitialStatusGatherer { suspend fun gatherInitialStatus(streams: TableCatalog): Map } diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDatabaseInitialStatus.kt similarity index 92% rename from airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt rename to airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDatabaseInitialStatus.kt index da942b77313d6..110f59f2dcbcc 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDestinationInitialStatus.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingDatabaseInitialStatus.kt @@ -4,15 +4,15 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping -import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatus +import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatus import java.time.Instant -data class TypingDedupingDestinationInitialStatus( +data class TypingDedupingDatabaseInitialStatus( /** Initial status of the final table, or null if the table doesn't exist yet. */ val finalTableStatus: FinalTableInitialStatus?, val rawTableStatus: RawTableInitialStatus?, val tempRawTableStatus: RawTableInitialStatus?, -) : DestinationInitialStatus +) : DatabaseInitialStatus data class FinalTableInitialStatus( val isSchemaMismatch: Boolean, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 07047dec1e5df..1d7df83e0b78c 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -18,7 +18,7 @@ private val logger = KotlinLogging.logger {} class TypingDedupingStreamLoader( override val stream: DestinationStream, - private val initialStatus: TypingDedupingDestinationInitialStatus, + private val initialStatus: TypingDedupingDatabaseInitialStatus, private val tableNames: TableNames, private val columnNameMapping: ColumnNameMapping, private val rawTableOperations: TypingDedupingRawTableOperations, diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index 0d81792fe37e7..18b2dd3c18a80 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -6,7 +6,7 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.orchestration.db.DatabaseHandler -import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader import java.util.concurrent.Executors @@ -16,15 +16,14 @@ import kotlinx.coroutines.runBlocking class TypingDedupingWriter( private val names: TableCatalog, - private val stateGatherer: - DestinationInitialStatusGatherer, + private val stateGatherer: DatabaseInitialStatusGatherer, private val databaseHandler: DatabaseHandler, private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, ) : DestinationWriter { private lateinit var initialStatuses: - Map + Map override suspend fun setup() { Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> @@ -34,7 +33,7 @@ class TypingDedupingWriter( databaseHandler.createNamespaces(namespaces.toSet()) val initialInitialStatuses: - Map = + Map = stateGatherer.gatherInitialStatus(names) // TODO migrations - we should probably actually drop all existing migrations as part of diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt similarity index 96% rename from airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt rename to airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt index bb8d9f134c599..45bd2c1cc330a 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDestinationInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt @@ -20,14 +20,14 @@ import io.airbyte.cdk.load.command.Overwrite import io.airbyte.cdk.load.data.ObjectType import io.airbyte.cdk.load.message.Meta import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping -import io.airbyte.cdk.load.orchestration.db.DestinationInitialStatusGatherer +import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer import io.airbyte.cdk.load.orchestration.db.TableName import io.airbyte.cdk.load.orchestration.db.TableNames import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.AlterTableReport import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.FinalTableInitialStatus import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.RawTableInitialStatus import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog -import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingDestinationInitialStatus +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingDatabaseInitialStatus import io.airbyte.cdk.util.CollectionUtils.containsAllIgnoreCase import io.airbyte.cdk.util.CollectionUtils.containsIgnoreCase import io.airbyte.cdk.util.CollectionUtils.matchingKey @@ -38,8 +38,8 @@ import java.util.stream.Stream private val logger = KotlinLogging.logger {} -class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : - DestinationInitialStatusGatherer { +class BigqueryDatabaseInitialStatusGatherer(private val bq: BigQuery) : + DatabaseInitialStatusGatherer { private fun findExistingTable(finalTableName: TableName): TableDefinition? { val table = bq.getTable(finalTableName.namespace, finalTableName.name) return table?.getDefinition() @@ -116,7 +116,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : override suspend fun gatherInitialStatus( streams: TableCatalog, - ): Map { + ): Map { return streams.mapValues { (stream, names) -> val (tableNames, columnNameMapping) = names val finalTable = findExistingTable(tableNames.finalTableName!!) @@ -143,7 +143,7 @@ class BigqueryDestinationInitialStatusGatherer(private val bq: BigQuery) : tableNames.rawTableName!!, TableNames.TMP_TABLE_SUFFIX, ) - TypingDedupingDestinationInitialStatus( + TypingDedupingDatabaseInitialStatus( finalTableStatus, rawTableState, tempRawTableState, diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index f02713c0e6e82..3e6075557a364 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -11,7 +11,7 @@ import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupin import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator -import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDestinationInitialStatusGatherer +import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDatabaseInitialStatusGatherer import io.micronaut.context.annotation.Factory import jakarta.inject.Singleton @@ -26,7 +26,7 @@ class BigqueryWriterFactory( val destinationHandler = BigQueryDatabaseHandler(bigquery, config.datasetLocation.region) return TypingDedupingWriter( names, - BigqueryDestinationInitialStatusGatherer(bigquery), + BigqueryDatabaseInitialStatusGatherer(bigquery), destinationHandler, BigqueryRawTableOperations(bigquery), TypingDedupingFinalTableOperations( From e2a1ea7d2ff218220321bb166614d7bbe9527ea8 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 10:21:20 -0700 Subject: [PATCH 44/55] derp --- .../db/legacy_typing_deduping/TypingDedupingWriter.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index 18b2dd3c18a80..5941231effe9f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -63,7 +63,7 @@ class TypingDedupingWriter( // Refetch their statuses. val statusesAfterSoftReset = stateGatherer.gatherInitialStatus( - names.filterKeys { streamsNeedingSoftReset.containsKey(it) } + TableCatalog(names.filterKeys { streamsNeedingSoftReset.containsKey(it) }) ) // second map "wins" when adding two maps together, so we'll retain the newer statuses. initialStatuses = initialInitialStatuses + statusesAfterSoftReset From edfa43c2df8925efa57006cfe1c08c799ffe6931 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 10:37:20 -0700 Subject: [PATCH 45/55] fix stuff --- .../legacy_typing_deduping/TableNameMapFactory.kt | 6 ++---- .../bigquery/formatter/BigQueryRecordFormatter.kt | 13 +++++-------- 2 files changed, 7 insertions(+), 12 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt index 8f86d5aed4e76..2a8eb4f83952a 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TableNameMapFactory.kt @@ -16,12 +16,10 @@ import javax.inject.Singleton data class TableNameInfo(val tableNames: TableNames, val columnNameMapping: ColumnNameMapping) -@JvmInline -value class TableCatalog(private val catalog: Map) : +data class TableCatalog(private val catalog: Map) : Map by catalog -@JvmInline -value class TableCatalogByDescriptor( +data class TableCatalogByDescriptor( private val catalog: Map ) : Map by catalog diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt index 6899485b0e404..250a18cb2bdaa 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/formatter/BigQueryRecordFormatter.kt @@ -71,14 +71,11 @@ class BigQueryRecordFormatter { // omitting the COLUMN_NAME_AB_LOADED_AT field and by rearranging the column order. val CSV_SCHEMA: Schema = Schema.of( - Field.of(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING), - Field.of( - JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT, - StandardSQLTypeName.TIMESTAMP - ), - Field.of(JavaBaseConstants.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING), - Field.of(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64), - Field.of(JavaBaseConstants.COLUMN_NAME_DATA, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP), + Field.of(Meta.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING), + Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64), + Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING), ) } } From f9a7794c14099430edb0c05aa9500f2f4644a504 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 11:01:57 -0700 Subject: [PATCH 46/55] pass state through? --- .../TypingDedupingExecutionConfig.kt | 9 +++++++++ .../legacy_typing_deduping/TypingDedupingStreamLoader.kt | 7 +++++++ .../db/legacy_typing_deduping/TypingDedupingWriter.kt | 3 +++ .../destination/bigquery/write/BigqueryWriter.kt | 4 ++++ .../bigquery/write/bulk_loader/BigQueryBulkLoader.kt | 9 +++++++-- 5 files changed, 30 insertions(+), 2 deletions(-) create mode 100644 airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingExecutionConfig.kt diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingExecutionConfig.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingExecutionConfig.kt new file mode 100644 index 0000000000000..4c1b43bdf1101 --- /dev/null +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingExecutionConfig.kt @@ -0,0 +1,9 @@ +/* + * Copyright (c) 2025 Airbyte, Inc., all rights reserved. + */ + +package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping + +data class TypingDedupingExecutionConfig( + val rawTableSuffix: String, +) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt index 1d7df83e0b78c..254965d72668f 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingStreamLoader.kt @@ -11,6 +11,7 @@ import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.NO_SUFFIX import io.airbyte.cdk.load.orchestration.db.TableNames.Companion.TMP_TABLE_SUFFIX import io.airbyte.cdk.load.state.StreamProcessingFailed import io.airbyte.cdk.load.write.StreamLoader +import io.airbyte.cdk.load.write.StreamStateStore import io.github.oshai.kotlinlogging.KotlinLogging import java.time.Instant @@ -24,6 +25,7 @@ class TypingDedupingStreamLoader( private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, + private val streamStateStore: StreamStateStore, ) : StreamLoader { private val isTruncateSync = when (stream.minimumGenerationId) { @@ -60,6 +62,11 @@ class TypingDedupingStreamLoader( logger.info { "Typing and deduping disabled, skipping final table initialization" } finalTmpTableSuffix = NO_SUFFIX } + + streamStateStore.put( + stream.descriptor, + TypingDedupingExecutionConfig(rawTableSuffix), + ) } private fun prepareStageForTruncate(): Pair { diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index 5941231effe9f..b2de5e554e559 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -9,6 +9,7 @@ import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer import io.airbyte.cdk.load.write.DestinationWriter import io.airbyte.cdk.load.write.StreamLoader +import io.airbyte.cdk.load.write.StreamStateStore import java.util.concurrent.Executors import kotlinx.coroutines.asCoroutineDispatcher import kotlinx.coroutines.launch @@ -21,6 +22,7 @@ class TypingDedupingWriter( private val rawTableOperations: TypingDedupingRawTableOperations, private val finalTableOperations: TypingDedupingFinalTableOperations, private val disableTypeDedupe: Boolean, + private val streamStateStore: StreamStateStore, ) : DestinationWriter { private lateinit var initialStatuses: Map @@ -80,6 +82,7 @@ class TypingDedupingWriter( rawTableOperations, finalTableOperations, disableTypeDedupe = disableTypeDedupe, + streamStateStore, ) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt index 3e6075557a364..c1473136f1d30 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/BigqueryWriter.kt @@ -6,8 +6,10 @@ package io.airbyte.integrations.destination.bigquery.write import com.google.cloud.bigquery.BigQuery import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingWriter +import io.airbyte.cdk.load.write.StreamStateStore import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDatabaseHandler import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator @@ -20,6 +22,7 @@ class BigqueryWriterFactory( private val bigquery: BigQuery, private val config: BigqueryConfiguration, private val names: TableCatalog, + private val streamStateStore: StreamStateStore, ) { @Singleton fun make(): TypingDedupingWriter { @@ -34,6 +37,7 @@ class BigqueryWriterFactory( destinationHandler, ), disableTypeDedupe = config.disableTypingDeduping, + streamStateStore ) } } diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt index 1f07757e83b19..7e4574c244858 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/bulk_loader/BigQueryBulkLoader.kt @@ -13,6 +13,8 @@ import io.airbyte.cdk.load.file.gcs.GcsClient import io.airbyte.cdk.load.message.StreamKey import io.airbyte.cdk.load.orchestration.db.TableName import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig +import io.airbyte.cdk.load.write.StreamStateStore import io.airbyte.cdk.load.write.db.BulkLoader import io.airbyte.cdk.load.write.db.BulkLoaderFactory import io.airbyte.integrations.destination.bigquery.BigQueryUtils @@ -30,9 +32,10 @@ class BigQueryBulkLoader( private val bigQueryClient: BigQuery, private val bigQueryConfiguration: BigqueryConfiguration, private val rawTableName: TableName, + private val rawTableSuffix: String, ) : BulkLoader { override suspend fun load(remoteObject: GcsBlob) { - val rawTableId = TableId.of(rawTableName.namespace, rawTableName.name) + val rawTableId = TableId.of(rawTableName.namespace, rawTableName.name + rawTableSuffix) val gcsUri = "gs://${remoteObject.storageConfig.gcsBucketName}/${remoteObject.key}" val csvOptions = @@ -86,7 +89,8 @@ class BigQueryBulkLoaderFactory( private val names: TableCatalogByDescriptor, private val storageClient: GcsClient, private val bigQueryClient: BigQuery, - private val bigQueryConfiguration: BigqueryConfiguration + private val bigQueryConfiguration: BigqueryConfiguration, + private val streamStateStore: StreamStateStore, ) : BulkLoaderFactory { override val numPartWorkers: Int = 2 override val numUploadWorkers: Int = 10 @@ -102,6 +106,7 @@ class BigQueryBulkLoaderFactory( bigQueryClient, bigQueryConfiguration, names[key.stream]!!.tableNames.rawTableName!!, + streamStateStore.get(key.stream)!!.rawTableSuffix, ) } } From 626a2f7aa6074bdaa61ad4ddf76eac05f664aef3 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Wed, 23 Apr 2025 11:08:23 -0700 Subject: [PATCH 47/55] whoops --- .../standard_insert/BigqueryBatchStandardInsertLoader.kt | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt index b495879ae0f44..828b9c0b293a4 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/write/standard_insert/BigqueryBatchStandardInsertLoader.kt @@ -16,8 +16,10 @@ import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.message.DestinationRecordRaw import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor +import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoaderFactory +import io.airbyte.cdk.load.write.StreamStateStore import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration @@ -68,14 +70,18 @@ class BigqueryBatchStandardInsertsLoaderFactory( private val bigquery: BigQuery, private val config: BigqueryConfiguration, private val names: TableCatalogByDescriptor, + private val streamStateStore: StreamStateStore, ) : DirectLoaderFactory { override fun create( streamDescriptor: DestinationStream.Descriptor, part: Int, ): BigqueryBatchStandardInsertsLoader { val tableName = names[streamDescriptor]!!.tableNames.rawTableName!! + val rawTableNameSuffix = streamStateStore.get(streamDescriptor)!!.rawTableSuffix val writeChannelConfiguration = - WriteChannelConfiguration.newBuilder(TableId.of(tableName.namespace, tableName.name)) + WriteChannelConfiguration.newBuilder( + TableId.of(tableName.namespace, tableName.name + rawTableNameSuffix) + ) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) .setSchema(BigQueryRecordFormatter.SCHEMA_V2) .setFormatOptions(FormatOptions.json()) From 7477f8562a1ab8e0beaa6bdd02c2fe40f12f905d Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:23:16 -0700 Subject: [PATCH 48/55] simplify raw table name gen --- .../TypingDedupingUtil.kt | 27 +++++++------------ 1 file changed, 10 insertions(+), 17 deletions(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt index 73da32e87318e..dadd9dd64e240 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingUtil.kt @@ -7,25 +7,18 @@ package io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping import kotlin.math.max object TypingDedupingUtil { - // copied wholesale from old CDK's StreamId fun concatenateRawTableName(namespace: String, name: String): String { val plainConcat = namespace + name - // Pretend we always have at least one underscore, so that we never generate - // `_raw_stream_` - var longestUnderscoreRun = 1 - var i = 0 - while (i < plainConcat.length) { - // If we've found an underscore, count the number of consecutive underscores - var underscoreRun = 0 - while (i < plainConcat.length && plainConcat[i] == '_') { - underscoreRun++ - i++ - } - longestUnderscoreRun = - max(longestUnderscoreRun.toDouble(), underscoreRun.toDouble()).toInt() - i++ - } + val longestUnderscoreRun = + Regex("_+") + .findAll(plainConcat) + .map { it.value.length } + .maxOrNull() + // Pretend we always have at least one underscore, so that we never + // generate `_raw_stream_` + .let { max(it ?: 0, 1) } - return namespace + "_raw" + "_".repeat(longestUnderscoreRun + 1) + "stream_" + name + val underscores = "_".repeat(longestUnderscoreRun + 1) + return "${namespace}_raw${underscores}stream_$name" } } From 3498570281b94f23dbba14472b0e942612a16442 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:24:23 -0700 Subject: [PATCH 49/55] update thread count --- .../db/legacy_typing_deduping/TypingDedupingWriter.kt | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt index b2de5e554e559..01df269f21469 100644 --- a/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt +++ b/airbyte-cdk/bulk/toolkits/load-db/src/main/kotlin/io/airbyte/cdk/load/orchestration/db/legacy_typing_deduping/TypingDedupingWriter.kt @@ -28,7 +28,7 @@ class TypingDedupingWriter( Map override suspend fun setup() { - Executors.newFixedThreadPool(4).asCoroutineDispatcher().use { dispatcher -> + Executors.newFixedThreadPool(10).asCoroutineDispatcher().use { dispatcher -> val namespaces = names.values.map { (tableNames, _) -> tableNames.rawTableName!!.namespace } + names.values.map { (tableNames, _) -> tableNames.finalTableName!!.namespace } From 22078b9eadb7736ade482281214c9907597d9b9f Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:26:07 -0700 Subject: [PATCH 50/55] trimIndent --- .../integrations/destination/bigquery/BigQueryUtils.kt | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt index 6a3e297ae2674..8868afc9221a3 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/BigQueryUtils.kt @@ -131,9 +131,9 @@ object BigQueryUtils { BigQueryConsts.FORMAT, Jsons.readTree( """{ - "format_type": "CSV", - "flattening": "No flattening" - }""" + "format_type": "CSV", + "flattening": "No flattening" + }""".trimIndent() ) ) .build() From 522bd19f67c78fd60ddfbdbb0d06500f3022671c Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:29:09 -0700 Subject: [PATCH 51/55] simplify impl --- .../bigquery/typing_deduping/BigQueryDatabaseHandler.kt | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt index 5b221a0b4d82c..e637fe23fdfb4 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt @@ -12,7 +12,6 @@ import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.JobStatistics import com.google.cloud.bigquery.JobStatus import com.google.cloud.bigquery.QueryJobConfiguration -import com.google.common.collect.Streams import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.Sql @@ -20,7 +19,6 @@ import io.airbyte.cdk.util.ConnectorExceptionUtil import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.github.oshai.kotlinlogging.KotlinLogging import java.util.UUID -import java.util.stream.Stream import kotlin.math.min import kotlinx.coroutines.coroutineScope import kotlinx.coroutines.launch @@ -55,11 +53,8 @@ class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocat Thread.sleep(1000L) job = job.reload() } - if (job.status.error != null) { - throw BigQueryException( - Streams.concat(Stream.of(job.status.error), job.status.executionErrors.stream()) - .toList() - ) + job.status.error?.let { + throw BigQueryException(listOf(job.status.error) + job.status.executionErrors) } val statistics = job.getStatistics() From c6c1f3aaf3e20010b909065e9578ed07151549d4 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:30:59 -0700 Subject: [PATCH 52/55] use kotlin stuff --- .../bigquery/typing_deduping/BigQueryDatabaseHandler.kt | 8 ++------ 1 file changed, 2 insertions(+), 6 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt index e637fe23fdfb4..36247773a3e13 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt @@ -68,12 +68,8 @@ class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocat // There isn't (afaict) anything resembling job.getChildJobs(), so we have to ask bq for // them bq.listJobs(BigQuery.JobListOption.parentJobId(job.jobId.job)) - .streamAll() - .sorted( - Comparator.comparing { childJob: Job -> - childJob.getStatistics().endTime - } - ) + .iterateAll() + .sortedBy { it.getStatistics().endTime } .forEach { childJob: Job -> val configuration = childJob.getConfiguration() if (configuration is QueryJobConfiguration) { From b1bacbfa59800ade3e22650f36b9d7a3f3f06c1f Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:32:35 -0700 Subject: [PATCH 53/55] better string munge --- .../bigquery/typing_deduping/BigQueryDatabaseHandler.kt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt index 36247773a3e13..5fbcdf8cd162a 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt @@ -77,8 +77,7 @@ class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocat childJob.getStatistics() var truncatedQuery: String = configuration.query - .replace("\n".toRegex(), " ") - .replace(" +".toRegex(), " ") + .replace("\\s+".toRegex(), " ") .substring( 0, min(100.0, configuration.query.length.toDouble()).toInt() From 1db68c5e72ad1d742c2e4efc692c157a7ec21f92 Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:34:31 -0700 Subject: [PATCH 54/55] use QueryJobConfiguration.of --- .../BigQueryDatabaseHandler.kt | 2 +- .../BigqueryDatabaseInitialStatusGatherer.kt | 28 +++++++++---------- 2 files changed, 14 insertions(+), 16 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt index 5fbcdf8cd162a..c40420b46ca95 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigQueryDatabaseHandler.kt @@ -45,7 +45,7 @@ class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocat bq.create( JobInfo.of( JobId.newBuilder().setLocation(datasetLocation).build(), - QueryJobConfiguration.newBuilder(statement).build() + QueryJobConfiguration.of(statement) ) ) // job.waitFor() gets stuck forever in some failure cases, so manually poll the job instead. diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt index 45bd2c1cc330a..1a621149b7e3d 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt @@ -61,14 +61,13 @@ class BigqueryDatabaseInitialStatusGatherer(private val bq: BigQuery) : val rawTableIdQuoted = """`${rawTableName.namespace}`.`${rawTableName.name}$suffix`""" val unloadedRecordTimestamp = bq.query( - QueryJobConfiguration.newBuilder( - """ - SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND) - FROM $rawTableIdQuoted - WHERE _airbyte_loaded_at IS NULL - """.trimIndent() - ) - .build() + QueryJobConfiguration.of( + """ + SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND) + FROM $rawTableIdQuoted + WHERE _airbyte_loaded_at IS NULL + """.trimIndent() + ) ) .iterateAll() .iterator() @@ -86,13 +85,12 @@ class BigqueryDatabaseInitialStatusGatherer(private val bq: BigQuery) : val loadedRecordTimestamp = bq.query( - QueryJobConfiguration.newBuilder( - """ - SELECT MAX(_airbyte_extracted_at) - FROM $rawTableIdQuoted - """.trimIndent() - ) - .build() + QueryJobConfiguration.of( + """ + SELECT MAX(_airbyte_extracted_at) + FROM $rawTableIdQuoted + """.trimIndent() + ) ) .iterateAll() .iterator() From 2f8e95e733e80fc7c192b4ccc0c45767f42b547e Mon Sep 17 00:00:00 2001 From: Edward Gao Date: Thu, 24 Apr 2025 09:35:26 -0700 Subject: [PATCH 55/55] merge filters --- .../typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt index 1a621149b7e3d..d2dbd94ba4028 100644 --- a/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt +++ b/airbyte-integrations/connectors/destination-bigquery/src/main/kotlin/io/airbyte/integrations/destination/bigquery/typing_deduping/BigqueryDatabaseInitialStatusGatherer.kt @@ -224,8 +224,7 @@ class BigqueryDatabaseInitialStatusGatherer(private val bq: BigQuery) : existingTable.schema!! .fields .stream() - .filter { field: Field -> pks.contains(field.name) } - .filter { field: Field -> field.mode == Field.Mode.REQUIRED } + .filter { pks.contains(it.name) && it.mode == Field.Mode.REQUIRED } .map { obj: Field -> obj.name } ) .collect(Collectors.toSet())