1
0
mirror of synced 2025-12-19 18:14:56 -05:00

Destination bigquery: Implement direct load (#59752)

Co-authored-by: Francis Genet <francis.genet@airbyte.io>
Co-authored-by: Octavia Squidington III <octavia-squidington-iii@users.noreply.github.com>
Co-authored-by: Ian Alton <ian.alton@airbyte.io>
This commit is contained in:
Edward Gao
2025-06-25 13:09:17 -07:00
committed by GitHub
parent f31bccd99e
commit e137a10160
34 changed files with 1794 additions and 1396 deletions

View File

@@ -0,0 +1,36 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.cdk.load.util
import java.math.BigDecimal
import kotlin.test.assertEquals
import org.junit.jupiter.api.Test
class BigDecimalUtilTest {
@Test
fun testMaxForRange() {
assertEquals(
BigDecimal("999.99"),
// 5 significant figures; 2 decimal points
BigDecimalUtil().maxForRange(precision = 5, scale = 2),
)
}
@Test
fun testNormalizedPrecision() {
assertEquals(6, BigDecimal("123.456").normalizedPrecision())
assertEquals(3, BigDecimal("123").normalizedPrecision())
// precision() = 3 (b/c BigDecimal represents this as 123 * 1000)
assertEquals(6, BigDecimal("1.23E5").normalizedPrecision())
}
@Test
fun testNormalizedScale() {
assertEquals(3, BigDecimal("123.456").normalizedScale())
assertEquals(0, BigDecimal("123").normalizedScale())
// scale = -3 (b/c BigDecimal represents this as 123 * 1000)
assertEquals(0, BigDecimal("1.23E5").normalizedScale())
}
}

View File

@@ -6,7 +6,7 @@ data:
connectorSubtype: database connectorSubtype: database
connectorType: destination connectorType: destination
definitionId: 22f6c74f-5699-40ff-833c-4a879ea40133 definitionId: 22f6c74f-5699-40ff-833c-4a879ea40133
dockerImageTag: 2.12.4 dockerImageTag: 3.0.0
dockerRepository: airbyte/destination-bigquery dockerRepository: airbyte/destination-bigquery
documentationUrl: https://docs.airbyte.com/integrations/destinations/bigquery documentationUrl: https://docs.airbyte.com/integrations/destinations/bigquery
githubIssueLabel: destination-bigquery githubIssueLabel: destination-bigquery
@@ -26,6 +26,9 @@ data:
2.0.0: 2.0.0:
message: "**Do not upgrade until you have run a test upgrade as outlined [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#testing-destinations-v2-for-a-single-connection)**.\nThis version introduces [Destinations V2](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#what-is-destinations-v2), which provides better error handling, incremental delivery of data for large syncs, and improved final table structures. To review the breaking changes, and how to upgrade, see [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#quick-start-to-upgrading). These changes will likely require updates to downstream dbt / SQL models, which we walk through [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#updating-downstream-transformations).\nSelecting `Upgrade` will upgrade **all** connections using this destination at their next sync. You can manually sync existing connections prior to the next scheduled sync to start the upgrade early.\n" message: "**Do not upgrade until you have run a test upgrade as outlined [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#testing-destinations-v2-for-a-single-connection)**.\nThis version introduces [Destinations V2](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#what-is-destinations-v2), which provides better error handling, incremental delivery of data for large syncs, and improved final table structures. To review the breaking changes, and how to upgrade, see [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#quick-start-to-upgrading). These changes will likely require updates to downstream dbt / SQL models, which we walk through [here](https://docs.airbyte.com/release_notes/upgrading_to_destinations_v2/#updating-downstream-transformations).\nSelecting `Upgrade` will upgrade **all** connections using this destination at their next sync. You can manually sync existing connections prior to the next scheduled sync to start the upgrade early.\n"
upgradeDeadline: "2023-11-07" upgradeDeadline: "2023-11-07"
3.0.0:
message: "If you never interact with the raw tables, you can upgrade without taking any action. Otherwise, make sure to read the migration guide for more details."
upgradeDeadline: "2026-07-31"
rolloutConfiguration: rolloutConfiguration:
enableProgressiveRollout: false enableProgressiveRollout: false
resourceRequirements: resourceRequirements:

View File

@@ -4,12 +4,10 @@
package io.airbyte.integrations.destination.bigquery package io.airbyte.integrations.destination.bigquery
object BigQueryConsts { object BigQueryConsts {
const val MiB: Int = 1024 * 1024
const val CONFIG_DATASET_ID: String = "dataset_id" const val CONFIG_DATASET_ID: String = "dataset_id"
const val CONFIG_PROJECT_ID: String = "project_id" const val CONFIG_PROJECT_ID: String = "project_id"
const val CONFIG_DATASET_LOCATION: String = "dataset_location" const val CONFIG_DATASET_LOCATION: String = "dataset_location"
const val CONFIG_CREDS: String = "credentials_json" const val CONFIG_CREDS: String = "credentials_json"
const val BIG_QUERY_CLIENT_CHUNK_SIZE: String = "big_query_client_buffer_size_mb"
const val LOADING_METHOD: String = "loading_method" const val LOADING_METHOD: String = "loading_method"
const val METHOD: String = "method" const val METHOD: String = "method"
@@ -19,9 +17,9 @@ object BigQueryConsts {
const val GCS_BUCKET_REGION: String = "gcs_bucket_region" const val GCS_BUCKET_REGION: String = "gcs_bucket_region"
const val CREDENTIAL: String = "credential" const val CREDENTIAL: String = "credential"
const val FORMAT: String = "format" const val FORMAT: String = "format"
const val KEEP_GCS_FILES: String = "keep_files_in_gcs-bucket"
const val KEEP_GCS_FILES_VAL: String = "Keep all tmp files in GCS"
const val DISABLE_TYPE_DEDUPE: String = "disable_type_dedupe" const val DISABLE_TYPE_DEDUPE: String = "disable_type_dedupe"
const val RAW_DATA_DATASET = "raw_data_dataset" const val RAW_DATA_DATASET = "raw_data_dataset"
const val CDC_DELETION_MODE: String = "cdc_deletion_mode"
const val NAMESPACE_PREFIX: String = "n" const val NAMESPACE_PREFIX: String = "n"
const val NULL_MARKER: String = "\\N"
} }

View File

@@ -11,22 +11,31 @@ import com.google.cloud.bigquery.BigQueryOptions
import io.airbyte.cdk.load.check.DestinationCheckerSync import io.airbyte.cdk.load.check.DestinationCheckerSync
import io.airbyte.cdk.load.command.DestinationCatalog import io.airbyte.cdk.load.command.DestinationCatalog
import io.airbyte.cdk.load.command.DestinationConfiguration import io.airbyte.cdk.load.command.DestinationConfiguration
import io.airbyte.cdk.load.orchestration.db.DefaultTempTableNameGenerator
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DefaultDirectLoadTableSqlOperations
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableWriter
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.NoopTypingDedupingSqlGenerator
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog 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.TypingDedupingExecutionConfig
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations 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.orchestration.db.legacy_typing_deduping.TypingDedupingWriter
import io.airbyte.cdk.load.state.SyncManager import io.airbyte.cdk.load.state.SyncManager
import io.airbyte.cdk.load.task.DestinationTaskLauncher import io.airbyte.cdk.load.task.DestinationTaskLauncher
import io.airbyte.cdk.load.write.DestinationWriter
import io.airbyte.cdk.load.write.StreamStateStore import io.airbyte.cdk.load.write.StreamStateStore
import io.airbyte.cdk.load.write.WriteOperation import io.airbyte.cdk.load.write.WriteOperation
import io.airbyte.integrations.destination.bigquery.check.BigqueryCheckCleaner import io.airbyte.integrations.destination.bigquery.check.BigqueryCheckCleaner
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration 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.BigqueryDatabaseInitialStatusGatherer
import io.airbyte.integrations.destination.bigquery.write.BigqueryRawTableOperations
import io.airbyte.integrations.destination.bigquery.write.bulk_loader.BigqueryBulkLoadConfiguration import io.airbyte.integrations.destination.bigquery.write.bulk_loader.BigqueryBulkLoadConfiguration
import io.airbyte.integrations.destination.bigquery.write.bulk_loader.BigqueryConfiguredForBulkLoad import io.airbyte.integrations.destination.bigquery.write.bulk_loader.BigqueryConfiguredForBulkLoad
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigQueryDatabaseHandler
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadDatabaseInitialStatusGatherer
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadNativeTableOperations
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadSqlGenerator
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadSqlTableOperations
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.legacy_raw_tables.BigqueryRawTableOperations
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.legacy_raw_tables.BigqueryTypingDedupingDatabaseInitialStatusGatherer
import io.github.oshai.kotlinlogging.KotlinLogging import io.github.oshai.kotlinlogging.KotlinLogging
import io.micronaut.context.annotation.Factory import io.micronaut.context.annotation.Factory
import io.micronaut.context.annotation.Requires import io.micronaut.context.annotation.Requires
@@ -70,21 +79,69 @@ class BigqueryBeansFactory {
bigquery: BigQuery, bigquery: BigQuery,
config: BigqueryConfiguration, config: BigqueryConfiguration,
names: TableCatalog, names: TableCatalog,
streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>, // micronaut will only instantiate a single instance of StreamStateStore,
): TypingDedupingWriter { // so accept it as a * generic and cast as needed.
// we use a different type depending on whether we're in legacy raw tables vs
// direct-load tables mode.
streamStateStore: StreamStateStore<*>,
): DestinationWriter {
val destinationHandler = BigQueryDatabaseHandler(bigquery, config.datasetLocation.region) val destinationHandler = BigQueryDatabaseHandler(bigquery, config.datasetLocation.region)
return TypingDedupingWriter( if (config.legacyRawTablesOnly) {
names, // force smart cast
BigqueryDatabaseInitialStatusGatherer(bigquery), @Suppress("UNCHECKED_CAST")
destinationHandler, streamStateStore as StreamStateStore<TypingDedupingExecutionConfig>
BigqueryRawTableOperations(bigquery), return TypingDedupingWriter(
TypingDedupingFinalTableOperations( names,
BigQuerySqlGenerator(config.projectId, config.datasetLocation.region), BigqueryTypingDedupingDatabaseInitialStatusGatherer(bigquery),
destinationHandler, destinationHandler,
), BigqueryRawTableOperations(bigquery),
disableTypeDedupe = config.disableTypingDeduping, TypingDedupingFinalTableOperations(
streamStateStore, NoopTypingDedupingSqlGenerator,
) destinationHandler,
),
disableTypeDedupe = true,
streamStateStore = streamStateStore,
)
} else {
val sqlTableOperations =
BigqueryDirectLoadSqlTableOperations(
DefaultDirectLoadTableSqlOperations(
BigqueryDirectLoadSqlGenerator(
projectId = config.projectId,
cdcDeletionMode = config.cdcDeletionMode,
),
destinationHandler,
),
bigquery,
)
// force smart cast
@Suppress("UNCHECKED_CAST")
streamStateStore as StreamStateStore<DirectLoadTableExecutionConfig>
val tempTableNameGenerator =
DefaultTempTableNameGenerator(internalNamespace = config.internalTableDataset)
return DirectLoadTableWriter(
internalNamespace = config.internalTableDataset,
names = names,
stateGatherer =
BigqueryDirectLoadDatabaseInitialStatusGatherer(
bigquery,
tempTableNameGenerator
),
destinationHandler = destinationHandler,
nativeTableOperations =
BigqueryDirectLoadNativeTableOperations(
bigquery,
sqlTableOperations,
destinationHandler,
projectId = config.projectId,
tempTableNameGenerator,
),
sqlTableOperations = sqlTableOperations,
streamStateStore = streamStateStore,
tempTableNameGenerator,
)
}
} }
@Singleton @Singleton

View File

@@ -8,9 +8,9 @@ import io.airbyte.cdk.load.check.CheckCleaner
import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.integrations.destination.bigquery.BigqueryBeansFactory import io.airbyte.integrations.destination.bigquery.BigqueryBeansFactory
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryFinalTableNameGenerator
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryRawTableNameGenerator
import io.airbyte.integrations.destination.bigquery.typing_deduping.toTableId import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
class BigqueryCheckCleaner : CheckCleaner<BigqueryConfiguration> { class BigqueryCheckCleaner : CheckCleaner<BigqueryConfiguration> {
override fun cleanup(config: BigqueryConfiguration, stream: DestinationStream) { override fun cleanup(config: BigqueryConfiguration, stream: DestinationStream) {

View File

@@ -3,55 +3,133 @@
*/ */
package io.airbyte.integrations.destination.bigquery.formatter package io.airbyte.integrations.destination.bigquery.formatter
import com.fasterxml.jackson.databind.node.ObjectNode
import com.google.cloud.bigquery.Field import com.google.cloud.bigquery.Field
import com.google.cloud.bigquery.QueryParameterValue import com.google.cloud.bigquery.QueryParameterValue
import com.google.cloud.bigquery.Schema import com.google.cloud.bigquery.Schema
import com.google.cloud.bigquery.StandardSQLTypeName import com.google.cloud.bigquery.StandardSQLTypeName
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.data.DateType
import io.airbyte.cdk.load.data.DateValue
import io.airbyte.cdk.load.data.EnrichedAirbyteValue
import io.airbyte.cdk.load.data.IntegerType
import io.airbyte.cdk.load.data.IntegerValue import io.airbyte.cdk.load.data.IntegerValue
import io.airbyte.cdk.load.data.NullValue
import io.airbyte.cdk.load.data.NumberType
import io.airbyte.cdk.load.data.NumberValue
import io.airbyte.cdk.load.data.ObjectValue
import io.airbyte.cdk.load.data.StringValue import io.airbyte.cdk.load.data.StringValue
import io.airbyte.cdk.load.data.TimeTypeWithTimezone
import io.airbyte.cdk.load.data.TimeTypeWithoutTimezone
import io.airbyte.cdk.load.data.TimeWithTimezoneValue
import io.airbyte.cdk.load.data.TimeWithoutTimezoneValue
import io.airbyte.cdk.load.data.TimestampTypeWithTimezone
import io.airbyte.cdk.load.data.TimestampTypeWithoutTimezone
import io.airbyte.cdk.load.data.TimestampWithTimezoneValue
import io.airbyte.cdk.load.data.TimestampWithoutTimezoneValue
import io.airbyte.cdk.load.message.DestinationRecordRaw import io.airbyte.cdk.load.message.DestinationRecordRaw
import io.airbyte.cdk.load.message.Meta import io.airbyte.cdk.load.message.Meta
import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping
import io.airbyte.cdk.load.util.BigDecimalUtil
import io.airbyte.cdk.load.util.Jsons
import io.airbyte.cdk.load.util.serializeToString import io.airbyte.cdk.load.util.serializeToString
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadSqlGenerator
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Reason
import java.math.BigInteger
import java.math.RoundingMode
import java.time.LocalDate
import java.time.LocalDateTime
import java.time.OffsetDateTime
import java.time.format.DateTimeFormatter
import java.util.concurrent.TimeUnit import java.util.concurrent.TimeUnit
/** /**
* The class formats incoming JsonSchema and AirbyteRecord in order to be inline with a * The class formats incoming JsonSchema and AirbyteRecord in order to be inline with a
* corresponding uploader. * corresponding uploader.
*/ */
class BigQueryRecordFormatter { class BigQueryRecordFormatter(
private val columnNameMapping: ColumnNameMapping,
private val legacyRawTablesOnly: Boolean,
) {
fun formatRecord(record: DestinationRecordRaw): String { fun formatRecord(record: DestinationRecordRaw): String {
val enrichedRecord = record.asEnrichedDestinationRecordAirbyteValue() val enrichedRecord = record.asEnrichedDestinationRecordAirbyteValue()
val outputRecord = mutableMapOf<String, Any?>() val outputRecord = mutableMapOf<String, Any?>()
enrichedRecord.airbyteMetaFields.forEach { (key, value) -> val enrichedFieldsToIterate =
if (legacyRawTablesOnly) {
// in legacy raw tables mode, we only need to look at the airbyte fields.
// and we just dump the actual data fields into the output record
// as a JSON blob.
outputRecord[Meta.COLUMN_NAME_DATA] = record.asJsonRecord().serializeToString()
enrichedRecord.airbyteMetaFields
} else {
// but in direct-load mode, we do actually need to look at all the fields.
enrichedRecord.allTypedFields
}
enrichedFieldsToIterate.forEach { (key, value) ->
when (key) { when (key) {
Meta.COLUMN_NAME_AB_EXTRACTED_AT -> { Meta.COLUMN_NAME_AB_EXTRACTED_AT -> {
val extractedAtMillis = (value.abValue as IntegerValue).value.longValueExact() val extractedAtMillis = (value.abValue as IntegerValue).value.longValueExact()
outputRecord[key] = getExtractedAt(extractedAtMillis) outputRecord[key] = getExtractedAt(extractedAtMillis)
} }
Meta.COLUMN_NAME_AB_META -> { Meta.COLUMN_NAME_AB_META -> {
// TODO this is a hack for T+D, we should remove it for direct-load tables // do nothing for now - we'll be updating the meta field when we process
// we're using sourceMeta instead of airbyteMeta, because the latter // other fields in this record.
// includes changes in-connector type coercion // so we need to defer it until _after_ we process the entire record.
// and for raw tables, we only want changes that originated from the source
val protocolMeta = enrichedRecord.sourceMeta.asProtocolObject()
protocolMeta.additionalProperties[Meta.AIRBYTE_META_SYNC_ID] =
record.stream.syncId
outputRecord[key] = protocolMeta.serializeToString()
// TODO we should do this for direct-load tables
// val serializedAirbyteMeta = (value.abValue as
// ObjectValue).serializeToString()
// outputRecord[key] = serializedAirbyteMeta
} }
Meta.COLUMN_NAME_AB_RAW_ID -> Meta.COLUMN_NAME_AB_RAW_ID ->
outputRecord[key] = (value.abValue as StringValue).value outputRecord[key] = (value.abValue as StringValue).value
Meta.COLUMN_NAME_AB_GENERATION_ID -> Meta.COLUMN_NAME_AB_GENERATION_ID ->
outputRecord[key] = (value.abValue as IntegerValue).value outputRecord[key] = (value.abValue as IntegerValue).value
else -> {
if (!legacyRawTablesOnly) {
// if we're null, then just don't write a value into the output JSON,
// so that bigquery will load a NULL value.
// Otherwise, do all the type validation stuff, then write a value into
// the output JSON.
if (value.abValue != NullValue) {
// first, validate the value.
validateAirbyteValue(value)
// then, populate the record.
// Bigquery has some strict requirements for datetime / time formatting,
// so handle that here.
when (value.type) {
TimestampTypeWithTimezone ->
outputRecord[columnNameMapping[key]!!] =
formatTimestampWithTimezone(value)
TimestampTypeWithoutTimezone ->
outputRecord[columnNameMapping[key]!!] =
formatTimestampWithoutTimezone(value)
TimeTypeWithoutTimezone ->
outputRecord[columnNameMapping[key]!!] =
formatTimeWithoutTimezone(value)
TimeTypeWithTimezone ->
outputRecord[columnNameMapping[key]!!] =
formatTimeWithTimezone(value)
else -> outputRecord[columnNameMapping[key]!!] = value.abValue
}
}
}
}
} }
} }
outputRecord[Meta.COLUMN_NAME_DATA] = record.asJsonRecord().serializeToString() // Now that we've gone through the whole record, we can process the airbyte_meta field.
outputRecord[Meta.COLUMN_NAME_AB_META] =
if (legacyRawTablesOnly) {
// this is a hack - in legacy mode, we don't do any in-connector validation
// so we just need to pass through the original record's airbyte_meta.
// so we completely ignore `value.abValue` here.
// (this is also probably hilariously slow, and it would be more efficient to just
// construct the string ourselves. but legacy raw tables isn't a mode we want to put
// a ton of effort into anyway)
val metaNode = Jsons.valueToTree(record.rawData.sourceMeta) as ObjectNode
metaNode.put("sync_id", record.stream.syncId)
metaNode.serializeToString()
} else {
(enrichedRecord.airbyteMeta.abValue as ObjectValue).values
}
return outputRecord.serializeToString() return outputRecord.serializeToString()
} }
@@ -65,6 +143,31 @@ class BigQueryRecordFormatter {
} }
companion object { companion object {
// see https://cloud.google.com/bigquery/docs/reference/standard-sql/data-types
private val INT64_MIN_VALUE: BigInteger = BigInteger.valueOf(Long.MIN_VALUE)
private val INT64_MAX_VALUE: BigInteger = BigInteger.valueOf(Long.MAX_VALUE)
private const val NUMERIC_MAX_PRECISION = 38
private const val NUMERIC_MAX_SCALE = 9
private val NUMERIC_MAX_VALUE =
BigDecimalUtil()
.maxForRange(precision = NUMERIC_MAX_PRECISION, scale = NUMERIC_MAX_SCALE)
private val NUMERIC_MIN_VALUE = NUMERIC_MAX_VALUE.negate()
private val DATE_MIN_VALUE = LocalDate.parse("0001-01-01")
private val DATE_MAX_VALUE = LocalDate.parse("9999-12-31")
private val TIMESTAMP_MIN_VALUE = OffsetDateTime.parse("0001-01-01T00:00:00Z")
private val TIMESTAMP_MAX_VALUE = OffsetDateTime.parse("9999-12-31T23:59:59.999999Z")
private val DATETIME_MIN_VALUE = LocalDateTime.parse("0001-01-01T00:00:00")
private val DATETIME_MAX_VALUE = LocalDateTime.parse("9999-12-31T23:59:59.999999")
private val DATETIME_WITH_TIMEZONE_FORMATTER: DateTimeFormatter =
DateTimeFormatter.ISO_OFFSET_DATE_TIME
private val DATETIME_WITHOUT_TIMEZONE_FORMATTER: DateTimeFormatter =
DateTimeFormatter.ISO_DATE_TIME
private val TIME_WITHOUT_TIMEZONE_FORMATTER: DateTimeFormatter =
DateTimeFormatter.ISO_LOCAL_TIME
private val TIME_WITH_TIMEZONE_FORMATTER: DateTimeFormatter =
DateTimeFormatter.ISO_OFFSET_TIME
// This is the schema used to represent the final raw table // This is the schema used to represent the final raw table
val SCHEMA_V2: Schema = val SCHEMA_V2: Schema =
Schema.of( Schema.of(
@@ -86,5 +189,122 @@ class BigQueryRecordFormatter {
Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64), Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64),
Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING), Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING),
) )
private val DIRECT_LOAD_SCHEMA =
listOf(
Field.newBuilder(Meta.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING)
.setMode(Field.Mode.REQUIRED)
.build(),
Field.newBuilder(Meta.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP)
.setMode(Field.Mode.REQUIRED)
.build(),
Field.newBuilder(Meta.COLUMN_NAME_AB_META, StandardSQLTypeName.JSON)
.setMode(Field.Mode.REQUIRED)
.build(),
Field.newBuilder(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64)
.setMode(Field.Mode.NULLABLE)
.build(),
)
fun getDirectLoadSchema(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
): Schema {
val userDefinedFields: List<Field> =
stream.schema
.asColumns()
.mapKeys { (originalName, _) -> columnNameMapping[originalName]!! }
.mapValues { (_, type) ->
BigqueryDirectLoadSqlGenerator.toDialectType(type.type)
}
.map { (name, type) -> Field.of(name, type) }
return Schema.of(DIRECT_LOAD_SCHEMA + userDefinedFields)
}
fun formatTimestampWithTimezone(value: EnrichedAirbyteValue): String {
return DATETIME_WITH_TIMEZONE_FORMATTER.format(
(value.abValue as TimestampWithTimezoneValue).value
)
}
fun formatTimestampWithoutTimezone(value: EnrichedAirbyteValue): String {
return DATETIME_WITHOUT_TIMEZONE_FORMATTER.format(
(value.abValue as TimestampWithoutTimezoneValue).value
)
}
fun formatTimeWithoutTimezone(value: EnrichedAirbyteValue): String {
return TIME_WITHOUT_TIMEZONE_FORMATTER.format(
(value.abValue as TimeWithoutTimezoneValue).value
)
}
fun formatTimeWithTimezone(value: EnrichedAirbyteValue): String {
return TIME_WITH_TIMEZONE_FORMATTER.format(
(value.abValue as TimeWithTimezoneValue).value
)
}
fun validateAirbyteValue(value: EnrichedAirbyteValue) {
when (value.type) {
is IntegerType -> {
(value.abValue as IntegerValue).value.let {
if (it < INT64_MIN_VALUE || INT64_MAX_VALUE < it) {
value.nullify(Reason.DESTINATION_FIELD_SIZE_LIMITATION)
}
}
}
is NumberType -> {
(value.abValue as NumberValue).value.let {
if (it < NUMERIC_MIN_VALUE || NUMERIC_MAX_VALUE < it) {
// If we're too large/small, then we have to null out.
value.nullify(Reason.DESTINATION_FIELD_SIZE_LIMITATION)
} else if (it.scale() > NUMERIC_MAX_SCALE) {
// But if we're within the min/max range, but have too many decimal
// points, then we can round off the number.
// experimentally, bigquery uses the half_up rounding strategy:
// select cast(json_query('{"foo": -0.0000000005}', "$.foo") as numeric)
// -> -0.000000001
// select cast(json_query('{"foo": 0.0000000005}', "$.foo") as numeric)
// -> 0.000000001
value.truncate(
NumberValue(it.setScale(NUMERIC_MAX_SCALE, RoundingMode.HALF_UP)),
Reason.DESTINATION_FIELD_SIZE_LIMITATION,
)
}
}
}
// NOTE: This validation is currently unreachable because our coercion logic in
// AirbyteValueCoercer already rejects date/time values outside supported ranges
// via DATE_TIME_FORMATTER and TIME_FORMATTER, the Meta change reason will therefore
// always be DESTINATION_SERIALIZATION_ERROR instead of
// DESTINATION_FIELD_SIZE_LIMITATION for now.
//
// However, we're planning to expand the supported date/time range in the coercion
// layer, which will make this validation relevant again. Keeping this code for
// that future change.
is DateType -> {
(value.abValue as DateValue).value.let {
if (it < DATE_MIN_VALUE || DATE_MAX_VALUE < it) {
value.nullify(Reason.DESTINATION_FIELD_SIZE_LIMITATION)
}
}
}
is TimestampTypeWithTimezone -> {
(value.abValue as TimestampWithTimezoneValue).value.let {
if (it < TIMESTAMP_MIN_VALUE || TIMESTAMP_MAX_VALUE < it) {
value.nullify(Reason.DESTINATION_FIELD_SIZE_LIMITATION)
}
}
}
is TimestampTypeWithoutTimezone -> {
(value.abValue as TimestampWithoutTimezoneValue).value.let {
if (it < DATETIME_MIN_VALUE || DATETIME_MAX_VALUE < it) {
value.nullify(Reason.DESTINATION_FIELD_SIZE_LIMITATION)
}
}
}
else -> {}
}
}
} }
} }

View File

@@ -16,9 +16,9 @@ data class BigqueryConfiguration(
val datasetId: String, val datasetId: String,
val loadingMethod: LoadingMethodConfiguration, val loadingMethod: LoadingMethodConfiguration,
val credentialsJson: String?, val credentialsJson: String?,
val transformationPriority: TransformationPriority, val cdcDeletionMode: CdcDeletionMode,
val rawTableDataset: String, val internalTableDataset: String,
val disableTypingDeduping: Boolean, val legacyRawTablesOnly: Boolean,
) : DestinationConfiguration() { ) : DestinationConfiguration() {
override val numOpenStreamWorkers = 3 override val numOpenStreamWorkers = 3
} }
@@ -54,14 +54,15 @@ class BigqueryConfigurationFactory :
datasetId = pojo.datasetId, datasetId = pojo.datasetId,
loadingMethodConfig, loadingMethodConfig,
credentialsJson = pojo.credentialsJson, credentialsJson = pojo.credentialsJson,
pojo.transformationPriority ?: TransformationPriority.INTERACTIVE, // default to hard delete for backwards compatibility.
rawTableDataset = cdcDeletionMode = pojo.cdcDeletionMode ?: CdcDeletionMode.HARD_DELETE,
if (pojo.rawTableDataset.isNullOrBlank()) { internalTableDataset =
if (pojo.internalTableDataset.isNullOrBlank()) {
DbConstants.DEFAULT_RAW_TABLE_NAMESPACE DbConstants.DEFAULT_RAW_TABLE_NAMESPACE
} else { } else {
pojo.rawTableDataset!! pojo.internalTableDataset!!
}, },
disableTypingDeduping = pojo.disableTypingDeduping ?: false, legacyRawTablesOnly = pojo.legacyRawTablesOnly ?: false,
) )
} }
} }

View File

@@ -75,31 +75,36 @@ class BigquerySpecification : ConfigurationSpecification() {
) )
val credentialsJson: String? = null val credentialsJson: String? = null
@get:JsonSchemaTitle("Transformation Query Run Type") @get:JsonSchemaTitle("CDC deletion mode")
@get:JsonPropertyDescription( @get:JsonPropertyDescription(
"""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 <a href="https://cloud.google.com/bigquery/docs/running-queries#queries">here</a>. 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 dont count towards your concurrent rate limit. Read more about batch queries <a href="https://cloud.google.com/bigquery/docs/running-queries#batch">here</a>. The default "interactive" value is used if not set explicitly.""", """Whether to execute CDC deletions as hard deletes (i.e. propagate source deletions to the destination), or soft deletes (i.e. leave a tombstone record in the destination). Defaults to hard deletes.""",
) )
@get:JsonProperty("transformation_priority", defaultValue = "interactive") // default hard delete for backwards compatibility
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 5}""") @get:JsonProperty("cdc_deletion_mode", defaultValue = "Hard delete")
val transformationPriority: TransformationPriority? = null @get:JsonSchemaInject(
json = """{"group": "sync_behavior", "order": 5, "always_show": true}""",
@get:JsonSchemaTitle("Raw Table Dataset Name")
@get:JsonPropertyDescription(
"""The dataset to write raw tables into (default: airbyte_internal)""",
) )
@get:JsonProperty("raw_data_dataset") val cdcDeletionMode: CdcDeletionMode? = null
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 7}""")
val rawTableDataset: String? = null
@get:JsonSchemaTitle( @get:JsonSchemaTitle(
"Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)", """Legacy raw tables""",
) )
@get:JsonPropertyDescription( @get:JsonPropertyDescription(
"""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""", """Write the legacy "raw tables" format, to enable backwards compatibility with older versions of this connector.""",
) )
// for compatibility with existing actor configs, we keep the old property name.
@get:JsonProperty("disable_type_dedupe") @get:JsonProperty("disable_type_dedupe")
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 8, "default": false}""") @get:JsonSchemaInject(json = """{"group": "advanced", "order": 7, "default": false}""")
val disableTypingDeduping: Boolean? = null val legacyRawTablesOnly: Boolean? = null
@get:JsonSchemaTitle("Airbyte Internal Table Dataset Name")
@get:JsonPropertyDescription(
"""Airbyte will use this dataset for various internal tables. In legacy raw tables mode, the raw tables will be stored in this dataset. Defaults to "airbyte_internal".""",
)
// for backwards compatibility, the JSON property is still called raw_data_dataset.
@get:JsonProperty("raw_data_dataset")
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 8}""")
val internalTableDataset: String? = null
} }
@JsonTypeInfo( @JsonTypeInfo(
@@ -204,6 +209,11 @@ enum class TransformationPriority(@get:JsonValue val transformationPriority: Str
BATCH("batch") BATCH("batch")
} }
enum class CdcDeletionMode(@get:JsonValue val cdcDeletionMode: String) {
HARD_DELETE("Hard delete"),
SOFT_DELETE("Soft delete"),
}
@Singleton @Singleton
class BigquerySpecificationExtension : DestinationSpecificationExtension { class BigquerySpecificationExtension : DestinationSpecificationExtension {
override val supportedSyncModes = override val supportedSyncModes =
@@ -216,6 +226,7 @@ class BigquerySpecificationExtension : DestinationSpecificationExtension {
override val groups = override val groups =
listOf( listOf(
DestinationSpecificationExtension.Group("connection", "Connection"), DestinationSpecificationExtension.Group("connection", "Connection"),
DestinationSpecificationExtension.Group("sync_behavior", "Sync Behavior"),
DestinationSpecificationExtension.Group("advanced", "Advanced"), DestinationSpecificationExtension.Group("advanced", "Advanced"),
) )
} }

View File

@@ -1,697 +0,0 @@
/*
* 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.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.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.*
import java.util.stream.Collectors
import org.apache.commons.lang3.StringUtils
/**
* @param projectId
* @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 {
private fun extractAndCast(
columnName: String,
airbyteType: AirbyteType,
forceSafeCast: Boolean
): String {
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)
}
val jsonPathEscapedColumnName = escapeColumnNameForJsonPath(columnName)
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
// 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`, '${'$'}."$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`, '${'$'}."$jsonPathEscapedColumnName"')
END, wide_number_mode=>'round')
""".trimIndent()
}
if (airbyteType.isArray) {
// Much like the Struct case above, arrays need special handling.
return """
PARSE_JSON(CASE
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`, '${'$'}."$jsonPathEscapedColumnName"')
END, wide_number_mode=>'round')
""".trimIndent()
}
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'), '${'$'}."$jsonPathEscapedColumnName"')"""
}
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.
return """
(CASE
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`, '${'$'}."$jsonPathEscapedColumnName"')
END)
""".trimIndent()
}
val dialectType = toDialectType(airbyteType)
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
} else {
// SAFE_CAST is actually a massive performance hit, so we should skip it if we can.
cast(baseTyping, dialectType.name, forceSafeCast)
}
}
override fun createFinalTable(
stream: DestinationStream,
tableName: TableName,
columnNameMapping: ColumnNameMapping,
finalTableSuffix: String,
replace: Boolean
): Sql {
val columnDeclarations = columnsAndTypes(stream, columnNameMapping)
val clusterConfig =
clusteringColumns(stream, columnNameMapping)
.stream()
.map { c: String? -> StringUtils.wrap(c, QUOTE) }
.collect(Collectors.joining(", "))
val finalTableId = tableName.toPrettyString(QUOTE, finalTableSuffix)
return Sql.separately(
listOfNotNull(
// CREATE OR REPLACE TABLE exists,
// but still returns an error if the new table has a different
// partitioning/clustering scheme.
// So we'll explicitly drop+create the table instead.
if (replace) {
"""DROP TABLE IF EXISTS `$projectId`.$finalTableId"""
} else {
null
},
"""
CREATE 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: DestinationStream,
columnNameMapping: ColumnNameMapping
): String {
return stream.schema
.asColumns()
.map { (fieldName, type) ->
val columnName = columnNameMapping[fieldName]!!
val typeName = toDialectType(type.type).name
"`$columnName` $typeName"
}
.joinToString(",\n")
}
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(tableNames.finalTableName!!, TableNames.SOFT_RESET_SUFFIX),
createFinalTable(
stream,
tableNames.finalTableName!!,
columnNameMapping,
TableNames.SOFT_RESET_SUFFIX,
true
),
clearLoadedAt(stream, tableNames.rawTableName!!)
)
}
private fun dropTableIfExists(
finalTableName: TableName,
suffix: String,
): Sql {
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.toPrettyString(QUOTE)
return Sql.of(
"""UPDATE `$projectId`.$rawTableId SET _airbyte_loaded_at = NULL WHERE 1=1;"""
)
}
override fun updateFinalTable(
stream: DestinationStream,
tableNames: TableNames,
columnNameMapping: ColumnNameMapping,
finalTableSuffix: String,
maxProcessedTimestamp: Instant?,
useExpensiveSaferCasting: Boolean,
): Sql {
val handleNewRecords =
if (stream.importType is Dedupe) {
upsertNewRecords(
stream,
tableNames,
columnNameMapping,
finalTableSuffix,
useExpensiveSaferCasting,
maxProcessedTimestamp
)
} else {
insertNewRecords(
stream,
tableNames,
columnNameMapping,
finalTableSuffix,
useExpensiveSaferCasting,
maxProcessedTimestamp
)
}
val commitRawTable = commitRawTable(tableNames.rawTableName!!, maxProcessedTimestamp)
return Sql.transactionally(handleNewRecords, commitRawTable)
}
private fun insertNewRecords(
stream: DestinationStream,
tableNames: TableNames,
columnNameMapping: ColumnNameMapping,
finalSuffix: String,
forceSafeCasting: Boolean,
minRawTimestamp: Instant?,
): String {
val columnList: String =
stream.schema
.asColumns()
.keys
.stream()
.map { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"`$columnName`,"
}
.collect(Collectors.joining("\n"))
val extractNewRawRecords =
extractNewRawRecords(
stream,
tableNames,
columnNameMapping,
forceSafeCasting,
minRawTimestamp
)
val finalTableId = tableNames.finalTableName!!.toPrettyString(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: DestinationStream,
tableNames: TableNames,
columnNameMapping: ColumnNameMapping,
finalSuffix: String,
forceSafeCasting: Boolean,
minRawTimestamp: Instant?,
): String {
val importType = stream.importType as Dedupe
val pkEquivalent =
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.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"`$columnName`,"
}
val newRecordColumnList: String =
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"new_record.`$columnName`,"
}
val extractNewRawRecords =
extractNewRawRecords(
stream,
tableNames,
columnNameMapping,
forceSafeCasting,
minRawTimestamp
)
val cursorComparison: String
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.
("""
(
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.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"
// 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.schema.asColumns().keys.joinToString("\n") { fieldName ->
val column = columnNameMapping[fieldName]!!
"`$column` = new_record.`$column`,"
}
val finalTableId = tableNames.finalTableName!!.toPrettyString(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: DestinationStream,
tableNames: TableNames,
columnNameMapping: ColumnNameMapping,
forceSafeCasting: Boolean,
minRawTimestamp: Instant?,
): String {
val columnCasts: String =
stream.schema
.asColumns()
.map { (fieldName, type) ->
val columnName = columnNameMapping[fieldName]!!
val extractAndCast = extractAndCast(fieldName, type.type, forceSafeCasting)
"$extractAndCast as `$columnName`,"
}
.joinToString("\n")
val columnErrors =
if (forceSafeCasting) {
"[" +
stream.schema
.asColumns()
.map { (fieldName, type) ->
val rawColName = escapeColumnNameForJsonPath(fieldName)
val jsonExtract = extractAndCast(fieldName, type.type, 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()
}
.joinToString(",\n") +
"]"
} else {
// We're not safe casting, so any error should throw an exception and trigger the
// safe cast logic
"[]"
}
val columnList: String =
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"`$columnName`,"
}
val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp)
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
// 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.schema.asColumns().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 =
importType.primaryKey.joinToString(",") { fieldName ->
val columnName = columnNameMapping[fieldName.first()]!!
"`$columnName`"
}
val cursorOrderClause =
if (importType.cursor.isEmpty()) {
""
} else if (importType.cursor.size == 1) {
val columnName = columnNameMapping[importType.cursor.first()]!!
"`$columnName` DESC NULLS LAST,"
} else {
throw UnsupportedOperationException(
"Only top-level cursors are supported, got ${importType.cursor}"
)
}
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(rawTableName: TableName, minRawTimestamp: Instant?): String {
val rawTableId = rawTableName.toPrettyString(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: DestinationStream,
finalTableName: TableName,
finalTableSuffix: String,
): Sql {
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}`;"
)
}
/**
* 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 = "`"
val nameTransformer = BigQuerySQLNameTransformer()
fun toDialectType(type: AirbyteType): StandardSQLTypeName =
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 -> StandardSQLTypeName.JSON
is UnionType ->
if (type.isLegacyUnion) {
toDialectType(type.chooseType())
} else {
StandardSQLTypeName.JSON
}
is UnknownType -> StandardSQLTypeName.JSON
}
fun clusteringColumns(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping
): List<String> {
val clusterColumns: MutableList<String> = ArrayList()
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.importType as Dedupe).primaryKey.stream().limit(3).forEach {
pk: List<String> ->
clusterColumns.add(columnNameMapping[pk.first()]!!)
}
}
clusterColumns.add("_airbyte_extracted_at")
return clusterColumns
}
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 {
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
}
}
}

View File

@@ -1,281 +0,0 @@
/*
* 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.ConfigErrorException
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.command.SoftDelete
import io.airbyte.cdk.load.command.Update
import io.airbyte.cdk.load.message.Meta
import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping
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.TypingDedupingDatabaseInitialStatus
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
import java.util.stream.Stream
private val logger = KotlinLogging.logger {}
class BigqueryDatabaseInitialStatusGatherer(private val bq: BigQuery) :
DatabaseInitialStatusGatherer<TypingDedupingDatabaseInitialStatus> {
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 null
val rawTableIdQuoted = """`${rawTableName.namespace}`.`${rawTableName.name}$suffix`"""
val unloadedRecordTimestamp =
bq.query(
QueryJobConfiguration.of(
"""
SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND)
FROM $rawTableIdQuoted
WHERE _airbyte_loaded_at IS NULL
""".trimIndent()
)
)
.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.of(
"""
SELECT MAX(_airbyte_extracted_at)
FROM $rawTableIdQuoted
""".trimIndent()
)
)
.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: TableCatalog,
): Map<DestinationStream, TypingDedupingDatabaseInitialStatus> {
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!!,
TableNames.TMP_TABLE_SUFFIX,
)
TypingDedupingDatabaseInitialStatus(
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
}
internal fun buildAlterTableReport(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
existingTable: TableDefinition,
): AlterTableReport {
val pks = getPks(stream, columnNameMapping)
val streamSchema: Map<String, StandardSQLTypeName> =
stream.schema.asColumns().entries.associate {
columnNameMapping[it.key]!! to BigQuerySqlGenerator.toDialectType(it.value.type)
}
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(Meta.COLUMN_NAMES, 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 { pks.contains(it.name) && it.mode == Field.Mode.REQUIRED }
.map { obj: Field -> obj.name }
)
.collect(Collectors.toSet())
return AlterTableReport(
columnsToAdd,
columnsToRemove,
columnsToChangeType,
)
}
companion object {
@VisibleForTesting
fun clusteringMatches(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
existingTable: StandardTableDefinition,
): Boolean {
return (existingTable.clustering != null &&
containsAllIgnoreCase(
HashSet<String>(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
}
private fun getPks(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping
): Set<String> {
return when (stream.importType) {
Append,
Overwrite -> emptySet()
is Dedupe ->
(stream.importType as Dedupe)
.primaryKey
.map { pk -> columnNameMapping[pk.first()]!! }
.toSet()
SoftDelete,
Update -> throw ConfigErrorException("Unsupported sync mode: ${stream.importType}")
}
}
}
}

View File

@@ -8,20 +8,23 @@ import com.google.cloud.bigquery.*
import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.BigQuery
import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.JobInfo
import com.google.cloud.bigquery.LoadJobConfiguration import com.google.cloud.bigquery.LoadJobConfiguration
import io.airbyte.cdk.load.command.DestinationCatalog
import io.airbyte.cdk.load.file.gcs.GcsBlob import io.airbyte.cdk.load.file.gcs.GcsBlob
import io.airbyte.cdk.load.file.gcs.GcsClient import io.airbyte.cdk.load.file.gcs.GcsClient
import io.airbyte.cdk.load.message.StreamKey import io.airbyte.cdk.load.message.StreamKey
import io.airbyte.cdk.load.orchestration.db.TableName import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor 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.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig
import io.airbyte.cdk.load.write.StreamStateStore import io.airbyte.cdk.load.write.StreamStateStore
import io.airbyte.cdk.load.write.db.BulkLoader import io.airbyte.cdk.load.write.db.BulkLoader
import io.airbyte.cdk.load.write.db.BulkLoaderFactory import io.airbyte.cdk.load.write.db.BulkLoaderFactory
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
import io.airbyte.integrations.destination.bigquery.BigQueryUtils import io.airbyte.integrations.destination.bigquery.BigQueryUtils
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing
import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
import io.micronaut.context.annotation.Requires import io.micronaut.context.annotation.Requires
import io.micronaut.context.condition.Condition import io.micronaut.context.condition.Condition
import io.micronaut.context.condition.ConditionContext import io.micronaut.context.condition.ConditionContext
@@ -31,11 +34,10 @@ class BigQueryBulkLoader(
private val storageClient: GcsClient, private val storageClient: GcsClient,
private val bigQueryClient: BigQuery, private val bigQueryClient: BigQuery,
private val bigQueryConfiguration: BigqueryConfiguration, private val bigQueryConfiguration: BigqueryConfiguration,
private val rawTableName: TableName, private val tableId: TableId,
private val rawTableSuffix: String, private val schema: Schema,
) : BulkLoader<GcsBlob> { ) : BulkLoader<GcsBlob> {
override suspend fun load(remoteObject: GcsBlob) { override suspend fun load(remoteObject: GcsBlob) {
val rawTableId = TableId.of(rawTableName.namespace, rawTableName.name + rawTableSuffix)
val gcsUri = "gs://${remoteObject.storageConfig.gcsBucketName}/${remoteObject.key}" val gcsUri = "gs://${remoteObject.storageConfig.gcsBucketName}/${remoteObject.key}"
val csvOptions = val csvOptions =
@@ -46,11 +48,12 @@ class BigQueryBulkLoader(
.build() .build()
val configuration = val configuration =
LoadJobConfiguration.builder(rawTableId, gcsUri) LoadJobConfiguration.builder(tableId, gcsUri)
.setFormatOptions(csvOptions) .setFormatOptions(csvOptions)
.setSchema(BigQueryRecordFormatter.CSV_SCHEMA) .setSchema(schema)
.setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND) .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
.setJobTimeoutMs(600000L) // 10 min timeout .setJobTimeoutMs(600000L) // 10 min timeout
.setNullMarker(BigQueryConsts.NULL_MARKER)
.build() .build()
val loadJob = bigQueryClient.create(JobInfo.of(configuration)) val loadJob = bigQueryClient.create(JobInfo.of(configuration))
@@ -59,7 +62,7 @@ class BigQueryBulkLoader(
BigQueryUtils.waitForJobFinish(loadJob) BigQueryUtils.waitForJobFinish(loadJob)
} catch (e: Exception) { } catch (e: Exception) {
throw RuntimeException( throw RuntimeException(
"Failed to load CSV data from $gcsUri to table ${rawTableId.dataset}.${rawTableId.table}: ${e.message}", "Failed to load CSV data from $gcsUri to table ${tableId.dataset}.${tableId.table}",
e e
) )
} }
@@ -86,11 +89,13 @@ class BigqueryConfiguredForBulkLoad : Condition {
@Singleton @Singleton
@Requires(condition = BigqueryConfiguredForBulkLoad::class) @Requires(condition = BigqueryConfiguredForBulkLoad::class)
class BigQueryBulkLoaderFactory( class BigQueryBulkLoaderFactory(
private val catalog: DestinationCatalog,
private val names: TableCatalogByDescriptor, private val names: TableCatalogByDescriptor,
private val storageClient: GcsClient, private val storageClient: GcsClient,
private val bigQueryClient: BigQuery, private val bigQueryClient: BigQuery,
private val bigQueryConfiguration: BigqueryConfiguration, private val bigQueryConfiguration: BigqueryConfiguration,
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>, private val typingDedupingStreamStateStore: StreamStateStore<TypingDedupingExecutionConfig>?,
private val directLoadStreamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>?,
) : BulkLoaderFactory<StreamKey, GcsBlob> { ) : BulkLoaderFactory<StreamKey, GcsBlob> {
override val numPartWorkers: Int = 2 override val numPartWorkers: Int = 2
override val numUploadWorkers: Int = 10 override val numUploadWorkers: Int = 10
@@ -101,12 +106,28 @@ class BigQueryBulkLoaderFactory(
override val maxMemoryRatioReservedForParts: Double = 0.6 override val maxMemoryRatioReservedForParts: Double = 0.6
override fun create(key: StreamKey, partition: Int): BulkLoader<GcsBlob> { override fun create(key: StreamKey, partition: Int): BulkLoader<GcsBlob> {
val tableId: TableId
val schema: Schema
val tableNameInfo = names[key.stream]!!
if (bigQueryConfiguration.legacyRawTablesOnly) {
val rawTableName = tableNameInfo.tableNames.rawTableName!!
val rawTableSuffix = typingDedupingStreamStateStore!!.get(key.stream)!!.rawTableSuffix
tableId = TableId.of(rawTableName.namespace, rawTableName.name + rawTableSuffix)
schema = BigQueryRecordFormatter.CSV_SCHEMA
} else {
tableId = directLoadStreamStateStore!!.get(key.stream)!!.tableName.toTableId()
schema =
BigQueryRecordFormatter.getDirectLoadSchema(
catalog.getStream(key.stream),
tableNameInfo.columnNameMapping,
)
}
return BigQueryBulkLoader( return BigQueryBulkLoader(
storageClient, storageClient,
bigQueryClient, bigQueryClient,
bigQueryConfiguration, bigQueryConfiguration,
names[key.stream]!!.tableNames.rawTableName!!, tableId,
streamStateStore.get(key.stream)!!.rawTableSuffix, schema,
) )
} }
} }

View File

@@ -42,7 +42,7 @@ data class BigqueryBulkLoadConfiguration(
ObjectStorageCompressionConfigurationProvider<BufferedOutputStream> { ObjectStorageCompressionConfigurationProvider<BufferedOutputStream> {
override val objectStoragePathConfiguration: ObjectStoragePathConfiguration override val objectStoragePathConfiguration: ObjectStoragePathConfiguration
override val objectStorageFormatConfiguration: ObjectStorageFormatConfiguration = override val objectStorageFormatConfiguration: ObjectStorageFormatConfiguration =
CSVFormatConfiguration() CSVFormatConfiguration(rootLevelFlattening = !bigQueryConfiguration.legacyRawTablesOnly)
override val objectStorageUploadConfiguration: ObjectStorageUploadConfiguration = override val objectStorageUploadConfiguration: ObjectStorageUploadConfiguration =
ObjectStorageUploadConfiguration() ObjectStorageUploadConfiguration()
override val s3BucketConfiguration: S3BucketConfiguration override val s3BucketConfiguration: S3BucketConfiguration

View File

@@ -0,0 +1,69 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.bulk_loader
import io.airbyte.cdk.load.data.ArrayType
import io.airbyte.cdk.load.data.NullValue
import io.airbyte.cdk.load.data.ObjectType
import io.airbyte.cdk.load.data.StringValue
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.data.csv.toCsvValue
import io.airbyte.cdk.load.message.DestinationRecordRaw
import io.airbyte.cdk.load.util.serializeToString
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter.Companion.formatTimeWithTimezone
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter.Companion.formatTimeWithoutTimezone
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter.Companion.formatTimestampWithTimezone
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter.Companion.formatTimestampWithoutTimezone
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter.Companion.validateAirbyteValue
class BigQueryCSVRowGenerator {
fun generate(record: DestinationRecordRaw, schema: ObjectType): List<Any> {
val enrichedRecord =
record.asEnrichedDestinationRecordAirbyteValue(
extractedAtAsTimestampWithTimezone = true
)
enrichedRecord.declaredFields.values.forEach { value ->
if (value.abValue is NullValue) {
return@forEach
}
validateAirbyteValue(value)
val actualValue = value.abValue
when (value.type) {
is TimestampTypeWithTimezone ->
value.abValue = StringValue(formatTimestampWithTimezone(value))
is TimestampTypeWithoutTimezone ->
value.abValue = StringValue(formatTimestampWithoutTimezone(value))
is TimeTypeWithTimezone ->
value.abValue = StringValue(formatTimeWithTimezone(value))
is TimeTypeWithoutTimezone ->
value.abValue = StringValue(formatTimeWithoutTimezone(value))
// serialize complex types to string
is ArrayType,
is ObjectType,
is UnionType,
is UnknownType -> value.abValue = StringValue(actualValue.serializeToString())
else -> {}
}
}
val values = enrichedRecord.allTypedFields
return schema.properties.map { (columnName, _) ->
val value = values[columnName]
if (value == null || value.abValue is NullValue) {
return@map BigQueryConsts.NULL_MARKER
}
value.abValue.toCsvValue()
}
}
}

View File

@@ -5,39 +5,53 @@
package io.airbyte.integrations.destination.bigquery.write.bulk_loader package io.airbyte.integrations.destination.bigquery.write.bulk_loader
import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.command.object_storage.ObjectStorageFormatConfigurationProvider import io.airbyte.cdk.load.data.withAirbyteMeta
import io.airbyte.cdk.load.file.csv.toCsvPrinterWithHeader
import io.airbyte.cdk.load.file.object_storage.CSVFormattingWriter import io.airbyte.cdk.load.file.object_storage.CSVFormattingWriter
import io.airbyte.cdk.load.file.object_storage.ObjectStorageFormattingWriter import io.airbyte.cdk.load.file.object_storage.ObjectStorageFormattingWriter
import io.airbyte.cdk.load.file.object_storage.ObjectStorageFormattingWriterFactory import io.airbyte.cdk.load.file.object_storage.ObjectStorageFormattingWriterFactory
import io.airbyte.cdk.load.message.DestinationRecordRaw import io.airbyte.cdk.load.message.DestinationRecordRaw
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import jakarta.inject.Singleton import jakarta.inject.Singleton
import java.io.OutputStream import java.io.OutputStream
class BigQueryObjectStorageFormattingWriter( class BigQueryObjectStorageFormattingWriter(
private val csvFormattingWriter: CSVFormattingWriter, stream: DestinationStream,
) : ObjectStorageFormattingWriter by csvFormattingWriter { outputStream: OutputStream,
) : ObjectStorageFormattingWriter {
private val finalSchema = stream.schema.withAirbyteMeta(true)
private val printer = finalSchema.toCsvPrinterWithHeader(outputStream)
private val bigQueryRowGenerator = BigQueryCSVRowGenerator()
override fun accept(record: DestinationRecordRaw) { override fun accept(record: DestinationRecordRaw) {
csvFormattingWriter.accept(record) printer.printRecord(bigQueryRowGenerator.generate(record, finalSchema))
}
override fun flush() {
printer.flush()
}
override fun close() {
printer.close()
} }
} }
@Singleton @Singleton
class BigQueryObjectStorageFormattingWriterFactory( class BigQueryObjectStorageFormattingWriterFactory(private val config: BigqueryConfiguration) :
private val formatConfigProvider: ObjectStorageFormatConfigurationProvider, ObjectStorageFormattingWriterFactory {
) : ObjectStorageFormattingWriterFactory {
override fun create( override fun create(
stream: DestinationStream, stream: DestinationStream,
outputStream: OutputStream outputStream: OutputStream,
): ObjectStorageFormattingWriter { ): ObjectStorageFormattingWriter {
val flatten = formatConfigProvider.objectStorageFormatConfiguration.rootLevelFlattening return if (config.legacyRawTablesOnly) {
return BigQueryObjectStorageFormattingWriter(
CSVFormattingWriter( CSVFormattingWriter(
stream, stream,
outputStream, outputStream,
rootLevelFlattening = flatten, rootLevelFlattening = false,
extractedAtAsTimestampWithTimezone = true, extractedAtAsTimestampWithTimezone = true,
), )
) } else {
BigQueryObjectStorageFormattingWriter(stream, outputStream)
}
} }
} }

View File

@@ -9,13 +9,16 @@ import com.google.cloud.bigquery.BigQueryException
import com.google.cloud.bigquery.FormatOptions import com.google.cloud.bigquery.FormatOptions
import com.google.cloud.bigquery.JobId import com.google.cloud.bigquery.JobId
import com.google.cloud.bigquery.JobInfo import com.google.cloud.bigquery.JobInfo
import com.google.cloud.bigquery.Schema
import com.google.cloud.bigquery.TableDataWriteChannel import com.google.cloud.bigquery.TableDataWriteChannel
import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.TableId
import com.google.cloud.bigquery.WriteChannelConfiguration import com.google.cloud.bigquery.WriteChannelConfiguration
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.ConfigErrorException
import io.airbyte.cdk.load.command.DestinationCatalog
import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.message.DestinationRecordRaw import io.airbyte.cdk.load.message.DestinationRecordRaw
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableExecutionConfig
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor 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.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig
import io.airbyte.cdk.load.write.DirectLoader import io.airbyte.cdk.load.write.DirectLoader
@@ -28,6 +31,7 @@ import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.CONFIG_ERROR_MSG import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.CONFIG_ERROR_MSG
import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.HTTP_STATUS_CODE_FORBIDDEN import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.HTTP_STATUS_CODE_FORBIDDEN
import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.HTTP_STATUS_CODE_NOT_FOUND import io.airbyte.integrations.destination.bigquery.write.standard_insert.BigqueryBatchStandardInsertsLoaderFactory.Companion.HTTP_STATUS_CODE_NOT_FOUND
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
import io.micronaut.context.annotation.Requires import io.micronaut.context.annotation.Requires
import io.micronaut.context.condition.Condition import io.micronaut.context.condition.Condition
import io.micronaut.context.condition.ConditionContext import io.micronaut.context.condition.ConditionContext
@@ -40,8 +44,8 @@ class BigqueryBatchStandardInsertsLoader(
private val bigquery: BigQuery, private val bigquery: BigQuery,
private val writeChannelConfiguration: WriteChannelConfiguration, private val writeChannelConfiguration: WriteChannelConfiguration,
private val job: JobId, private val job: JobId,
private val recordFormatter: BigQueryRecordFormatter,
) : DirectLoader { ) : DirectLoader {
private val recordFormatter = BigQueryRecordFormatter()
// a TableDataWriteChannel holds (by default) a 15MB buffer in memory. // a TableDataWriteChannel holds (by default) a 15MB buffer in memory.
// so we start out by writing to a BAOS, which grows dynamically. // so we start out by writing to a BAOS, which grows dynamically.
// when the BAOS reaches 15MB, we create the TableDataWriteChannel and switch over // when the BAOS reaches 15MB, we create the TableDataWriteChannel and switch over
@@ -115,24 +119,38 @@ class BigqueryConfiguredForBatchStandardInserts : Condition {
@Requires(condition = BigqueryConfiguredForBatchStandardInserts::class) @Requires(condition = BigqueryConfiguredForBatchStandardInserts::class)
@Singleton @Singleton
class BigqueryBatchStandardInsertsLoaderFactory( class BigqueryBatchStandardInsertsLoaderFactory(
private val catalog: DestinationCatalog,
private val bigquery: BigQuery, private val bigquery: BigQuery,
private val config: BigqueryConfiguration, private val config: BigqueryConfiguration,
private val tableCatalog: TableCatalogByDescriptor, private val tableCatalog: TableCatalogByDescriptor,
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>, private val typingDedupingStreamStateStore: StreamStateStore<TypingDedupingExecutionConfig>?,
private val directLoadStreamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>?,
) : DirectLoaderFactory<BigqueryBatchStandardInsertsLoader> { ) : DirectLoaderFactory<BigqueryBatchStandardInsertsLoader> {
override fun create( override fun create(
streamDescriptor: DestinationStream.Descriptor, streamDescriptor: DestinationStream.Descriptor,
part: Int, part: Int,
): BigqueryBatchStandardInsertsLoader { ): BigqueryBatchStandardInsertsLoader {
val rawTableName = tableCatalog[streamDescriptor]!!.tableNames.rawTableName!! val tableId: TableId
val rawTableNameSuffix = streamStateStore.get(streamDescriptor)!!.rawTableSuffix val schema: Schema
val tableNameInfo = tableCatalog[streamDescriptor]!!
val writeChannelConfiguration = if (config.legacyRawTablesOnly) {
WriteChannelConfiguration.newBuilder( val rawTableName = tableNameInfo.tableNames.rawTableName!!
TableId.of(rawTableName.namespace, rawTableName.name + rawTableNameSuffix) val rawTableSuffix =
typingDedupingStreamStateStore!!.get(streamDescriptor)!!.rawTableSuffix
tableId = TableId.of(rawTableName.namespace, rawTableName.name + rawTableSuffix)
schema = BigQueryRecordFormatter.SCHEMA_V2
} else {
tableId = directLoadStreamStateStore!!.get(streamDescriptor)!!.tableName.toTableId()
schema =
BigQueryRecordFormatter.getDirectLoadSchema(
catalog.getStream(streamDescriptor),
tableNameInfo.columnNameMapping,
) )
}
val writeChannelConfiguration =
WriteChannelConfiguration.newBuilder(tableId)
.setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED) .setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED)
.setSchema(BigQueryRecordFormatter.SCHEMA_V2) .setSchema(schema)
// new-line delimited json. // new-line delimited json.
.setFormatOptions(FormatOptions.json()) .setFormatOptions(FormatOptions.json())
.build() .build()
@@ -148,6 +166,10 @@ class BigqueryBatchStandardInsertsLoaderFactory(
bigquery, bigquery,
writeChannelConfiguration, writeChannelConfiguration,
jobId, jobId,
BigQueryRecordFormatter(
tableNameInfo.columnNameMapping,
legacyRawTablesOnly = config.legacyRawTablesOnly,
),
) )
} }

View File

@@ -1,7 +1,7 @@
/* /*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved. * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/ */
package io.airbyte.integrations.destination.bigquery.typing_deduping package io.airbyte.integrations.destination.bigquery.write.typing_deduping
import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.BigQuery
import com.google.cloud.bigquery.BigQueryException import com.google.cloud.bigquery.BigQueryException
@@ -12,6 +12,7 @@ import com.google.cloud.bigquery.JobInfo
import com.google.cloud.bigquery.JobStatistics import com.google.cloud.bigquery.JobStatistics
import com.google.cloud.bigquery.JobStatus import com.google.cloud.bigquery.JobStatus
import com.google.cloud.bigquery.QueryJobConfiguration import com.google.cloud.bigquery.QueryJobConfiguration
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.ConfigErrorException import io.airbyte.cdk.ConfigErrorException
import io.airbyte.cdk.load.orchestration.db.DatabaseHandler import io.airbyte.cdk.load.orchestration.db.DatabaseHandler
import io.airbyte.cdk.load.orchestration.db.Sql import io.airbyte.cdk.load.orchestration.db.Sql
@@ -21,12 +22,59 @@ import io.github.oshai.kotlinlogging.KotlinLogging
import java.util.UUID import java.util.UUID
import kotlin.math.min import kotlin.math.min
import kotlinx.coroutines.coroutineScope import kotlinx.coroutines.coroutineScope
import kotlinx.coroutines.delay
import kotlinx.coroutines.launch import kotlinx.coroutines.launch
private val logger = KotlinLogging.logger {} private val logger = KotlinLogging.logger {}
@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION", justification = "Kotlin is hard")
class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocation: String) : class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocation: String) :
DatabaseHandler { DatabaseHandler {
/**
* Some statements (e.g. ALTER TABLE) have strict rate limits. Bigquery recommends retrying
* these statements with exponential backoff, and the SDK doesn't do it automatically. So this
* function implements a basic retry loop.
*
* Technically, [statement] can contain multiple semicolon-separated statements. That's probably
* not a great idea (it's hard to reason about retrying partially-successful statements), so
* maybe don't do that. Just call this function multiple times.
*/
suspend fun executeWithRetries(
statement: String,
initialDelay: Long = 1000,
numAttempts: Int = 5,
maxDelay: Long = 60,
) {
var delay = initialDelay
for (attemptNumber in 1..numAttempts) {
try {
execute(Sql.of(statement))
return
} catch (e: Exception) {
// you might think that `e.isRetryable` would be useful here,
// and you would be wrong - presumably the SDK treats all 403 errors as
// nonretryable.
// instead, we hardcode handling for the rate-limit error... which requires matching
// against a specific magic string >.>
if (
e is BigQueryException && e.code == 403 && e.error.reason == "rateLimitExceeded"
) {
logger.warn(e) {
"Rate limit exceeded while executing SQL (attempt $attemptNumber/$numAttempts). Sleeping ${delay}ms and retrying."
}
val withJitter = delay + 1000 * Math.random()
delay(withJitter.toLong())
delay = min(delay * 2, maxDelay)
} else {
logger.error(e) {
"Caught exception while executing SQL (attempt $attemptNumber/$numAttempts). Not retrying."
}
throw e
}
}
}
}
@Throws(InterruptedException::class) @Throws(InterruptedException::class)
override fun execute(sql: Sql) { override fun execute(sql: Sql) {
val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION") val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION")

View File

@@ -2,7 +2,7 @@
* Copyright (c) 2025 Airbyte, Inc., all rights reserved. * Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/ */
package io.airbyte.integrations.destination.bigquery.typing_deduping package io.airbyte.integrations.destination.bigquery.write.typing_deduping
import com.google.cloud.bigquery.TableId import com.google.cloud.bigquery.TableId
import io.airbyte.cdk.load.command.DestinationStream import io.airbyte.cdk.load.command.DestinationStream
@@ -11,16 +11,18 @@ import io.airbyte.cdk.load.orchestration.db.FinalTableNameGenerator
import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator
import io.airbyte.cdk.load.orchestration.db.TableName import io.airbyte.cdk.load.orchestration.db.TableName
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingUtil import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingUtil
import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer
import java.util.Locale import java.util.Locale
import javax.inject.Singleton import javax.inject.Singleton
private val nameTransformer = BigQuerySQLNameTransformer()
@Singleton @Singleton
class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTableNameGenerator { class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTableNameGenerator {
override fun getTableName(streamDescriptor: DestinationStream.Descriptor) = override fun getTableName(streamDescriptor: DestinationStream.Descriptor) =
TableName( TableName(
nameTransformer.getNamespace(config.rawTableDataset), nameTransformer.getNamespace(config.internalTableDataset),
nameTransformer.convertStreamName( nameTransformer.convertStreamName(
TypingDedupingUtil.concatenateRawTableName( TypingDedupingUtil.concatenateRawTableName(
streamDescriptor.namespace ?: config.datasetId, streamDescriptor.namespace ?: config.datasetId,

View File

@@ -0,0 +1,50 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables
import com.google.cloud.bigquery.BigQuery
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.orchestration.db.DatabaseInitialStatusGatherer
import io.airbyte.cdk.load.orchestration.db.TableName
import io.airbyte.cdk.load.orchestration.db.TempTableNameGenerator
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadInitialStatus
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableStatus
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalog
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
import java.math.BigInteger
import java.util.concurrent.ConcurrentHashMap
import kotlinx.coroutines.coroutineScope
import kotlinx.coroutines.launch
@SuppressFBWarnings(value = ["NP_NONNULL_PARAM_VIOLATION"], justification = "kotlin coroutines")
class BigqueryDirectLoadDatabaseInitialStatusGatherer(
private val bigquery: BigQuery,
private val tempTableNameGenerator: TempTableNameGenerator,
) : DatabaseInitialStatusGatherer<DirectLoadInitialStatus> {
override suspend fun gatherInitialStatus(
streams: TableCatalog,
): Map<DestinationStream, DirectLoadInitialStatus> {
val map = ConcurrentHashMap<DestinationStream, DirectLoadInitialStatus>(streams.size)
coroutineScope {
streams.forEach { (stream, tableNameInfo) ->
launch {
val tableName = tableNameInfo.tableNames.finalTableName!!
map[stream] =
DirectLoadInitialStatus(
realTable = getTableStatus(tableName),
tempTable = getTableStatus(tempTableNameGenerator.generate(tableName)),
)
}
}
}
return map
}
private fun getTableStatus(tableName: TableName): DirectLoadTableStatus? {
val table = bigquery.getTable(tableName.toTableId())
return table?.let { DirectLoadTableStatus(isEmpty = table.numRows == BigInteger.ZERO) }
}
}

View File

@@ -0,0 +1,405 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables
import com.google.cloud.bigquery.BigQuery
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.TimePartitioning
import com.google.common.annotations.VisibleForTesting
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.ConfigErrorException
import io.airbyte.cdk.load.command.DestinationStream
import io.airbyte.cdk.load.message.Meta
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.TempTableNameGenerator
import io.airbyte.cdk.load.orchestration.db.direct_load_table.AlterTableReport
import io.airbyte.cdk.load.orchestration.db.direct_load_table.ColumnAdd
import io.airbyte.cdk.load.orchestration.db.direct_load_table.ColumnChange
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableNativeOperations
import io.airbyte.cdk.util.CollectionUtils.containsAllIgnoreCase
import io.airbyte.cdk.util.containsIgnoreCase
import io.airbyte.cdk.util.findIgnoreCase
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigQueryDatabaseHandler
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
import io.github.oshai.kotlinlogging.KotlinLogging
import kotlinx.coroutines.runBlocking
import org.apache.commons.codec.digest.DigestUtils
private val logger = KotlinLogging.logger {}
@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION", "kotlin coroutines")
class BigqueryDirectLoadNativeTableOperations(
private val bigquery: BigQuery,
private val sqlOperations: BigqueryDirectLoadSqlTableOperations,
private val databaseHandler: BigQueryDatabaseHandler,
private val projectId: String,
private val tempTableNameGenerator: TempTableNameGenerator,
) : DirectLoadTableNativeOperations {
override suspend fun ensureSchemaMatches(
stream: DestinationStream,
tableName: TableName,
columnNameMapping: ColumnNameMapping,
) {
val existingTable =
bigquery.getTable(tableName.toTableId()).getDefinition<TableDefinition>()
val shouldRecreateTable = shouldRecreateTable(stream, columnNameMapping, existingTable)
val alterTableReport = buildAlterTableReport(stream, columnNameMapping, existingTable)
logger.info {
"Stream ${stream.mappedDescriptor.toPrettyString()} had alter table report $alterTableReport"
}
try {
if (shouldRecreateTable) {
logger.info {
"Stream ${stream.mappedDescriptor.toPrettyString()} detected change in partitioning/clustering config. Recreating the table."
}
recreateTable(
stream,
columnNameMapping,
tableName,
alterTableReport.columnsToRetain,
alterTableReport.columnsToChangeType,
)
} else if (!alterTableReport.isNoOp) {
logger.info {
"Stream ${stream.mappedDescriptor.toPrettyString()} detected schema change. Altering the table."
}
runBlocking {
alterTable(
tableName,
columnsToAdd = alterTableReport.columnsToAdd,
columnsToRemove = alterTableReport.columnsToRemove,
columnsToChange = alterTableReport.columnsToChangeType,
)
}
} else {
logger.info {
"Stream ${stream.mappedDescriptor.toPrettyString()} has correct schema; no action needed."
}
}
} catch (e: Exception) {
logger.error(e) {
"Encountered an error while modifying the schema for stream ${stream.mappedDescriptor.toPrettyString()}. If this error persists, you may need to manually modify the table's schema."
}
throw e
}
}
override suspend fun getGenerationId(tableName: TableName): Long {
val result =
bigquery.query(
QueryJobConfiguration.of(
"SELECT _airbyte_generation_id FROM ${tableName.namespace}.${tableName.name} LIMIT 1",
),
)
val value = result.iterateAll().first().get(Meta.COLUMN_NAME_AB_GENERATION_ID)
return if (value.isNull) {
0
} else {
value.longValue
}
}
/**
* Bigquery doesn't support changing a table's partitioning / clustering scheme in-place. So
* check whether we want to change those here.
*/
private fun shouldRecreateTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
existingTable: TableDefinition
): Boolean {
var tableClusteringMatches = false
var tablePartitioningMatches = false
if (existingTable is StandardTableDefinition) {
tableClusteringMatches = clusteringMatches(stream, columnNameMapping, existingTable)
tablePartitioningMatches = partitioningMatches(existingTable)
}
return !tableClusteringMatches || !tablePartitioningMatches
}
internal fun buildAlterTableReport(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
existingTable: TableDefinition,
): AlterTableReport<StandardSQLTypeName> {
val expectedSchema: Map<String, StandardSQLTypeName> =
stream.schema.asColumns().entries.associate {
columnNameMapping[it.key]!! to
BigqueryDirectLoadSqlGenerator.toDialectType(it.value.type)
}
val actualSchema =
existingTable.schema!!.fields.associate { it.name to it.type.standardType }
// Columns in the StreamConfig that don't exist in the TableDefinition
val columnsToAdd =
expectedSchema
.filter { (name, _) -> actualSchema.findIgnoreCase(name) == null }
.map { (name, type) -> ColumnAdd(name, type) }
.toList()
// Columns in the current schema that are no longer in the DestinationStream
val columnsToRemove =
actualSchema.keys.filter { name ->
!expectedSchema.keys.containsIgnoreCase(name) &&
!Meta.COLUMN_NAMES.containsIgnoreCase(name)
}
// Columns that are typed differently than the DestinationStream
val columnsToChangeType =
expectedSchema.mapNotNull { (expectedName, expectedType) ->
actualSchema.findIgnoreCase(expectedName)?.let { actualType ->
if (actualType != expectedType) {
ColumnChange(
name = expectedName,
originalType = actualType,
newType = expectedType,
)
} else {
null
}
}
}
val columnsToRetain =
actualSchema.mapNotNull { (actualName, _) ->
if (
!columnsToRemove.contains(actualName) &&
!columnsToChangeType.any { it.name.equals(actualName, ignoreCase = true) }
) {
actualName
} else {
null
}
}
return AlterTableReport(
columnsToAdd = columnsToAdd,
columnsToRemove = columnsToRemove,
columnsToChangeType = columnsToChangeType,
columnsToRetain = columnsToRetain,
)
}
private fun getColumnCastStatement(
columnName: String,
originalType: StandardSQLTypeName,
newType: StandardSQLTypeName,
): String {
if (originalType == StandardSQLTypeName.JSON) {
// somewhat annoying.
// TO_JSON_STRING returns string values with double quotes, which is not what we want
// (i.e. we should unwrap the strings).
// but JSON_VALUE doesn't handle non-scalar values.
// so we have to handle both cases explicitly.
// there's technically some cases where this doesn't round-trip, e.g.
// JSON'"{\"foo\": 42}"' -> '{"foo":42}' -> JSON'{"foo": 42}'
// but that seems like a weird enough situation that we shouldn't worry about it.
return """
CAST(
CASE JSON_TYPE($columnName)
WHEN 'object' THEN TO_JSON_STRING($columnName)
WHEN 'array' THEN TO_JSON_STRING($columnName)
ELSE JSON_VALUE($columnName)
END
AS $newType
)
""".trimIndent()
} else if (newType == StandardSQLTypeName.JSON) {
return "TO_JSON($columnName)"
} else {
return "CAST($columnName AS $newType)"
}
}
/**
* roughly:
* 1. create a temp table
* 2. copy the existing data into it (casting columns as needed)
* 3. replace the real table with the temp table
*/
private suspend fun recreateTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
tableName: TableName,
columnsToRetain: List<String>,
columnsToChange: List<ColumnChange<StandardSQLTypeName>>,
) {
// can't just use the base temp table directly, since that could conflict with
// a truncate-refresh temp table.
// so add an explicit suffix that this is for schema change.
val tempTableName =
tempTableNameGenerator.generate(tableName).let {
it.copy(name = it.name + "_airbyte_tmp_schema_change")
}
val originalTableId = "`$projectId`.`${tableName.namespace}`.`${tableName.name}`"
val tempTableId = "`$projectId`.`${tempTableName.namespace}`.`${tempTableName.name}`"
val columnList =
(Meta.COLUMN_NAMES + columnsToRetain + columnsToChange.map { it.name }).joinToString(
",",
)
val valueList =
(Meta.COLUMN_NAMES +
columnsToRetain +
columnsToChange.map {
getColumnCastStatement(
columnName = it.name,
originalType = it.originalType,
newType = it.newType,
)
})
.joinToString(",")
// note: we don't care about columnsToDrop (because they don't exist in the tempTable)
// and we don't care about columnsToAdd (because they'll just default to null)
val insertToTempTable =
Sql.of(
"""
INSERT INTO $tempTableId
($columnList)
SELECT
$valueList
FROM $originalTableId
""".trimIndent(),
)
logger.info {
"Stream ${stream.mappedDescriptor.toPrettyString()} using temporary table ${tempTableName.toPrettyString()} to recreate table ${tableName.toPrettyString()}."
}
sqlOperations.createTable(
stream,
tempTableName,
columnNameMapping,
replace = true,
)
databaseHandler.execute(insertToTempTable)
sqlOperations.overwriteTable(tempTableName, tableName)
}
private suspend fun alterTable(
tableName: TableName,
columnsToAdd: List<ColumnAdd<StandardSQLTypeName>>,
columnsToRemove: List<String>,
columnsToChange: List<ColumnChange<StandardSQLTypeName>>,
) {
// the bigquery API only supports adding new fields; you can't drop/rename existing fields.
// so we'll do everything via DDL.
// We also try to batch operations into a single statement, because bigquery enforces
// somewhat low rate limits on how many ALTER TABLE operations you can run in a short
// timeframe.
val tableId = """`$projectId`.`${tableName.namespace}`.`${tableName.name}`"""
// bigquery has strict limits on what types can be altered to other types.
// so instead, we actually add a new column, explicitly cast the old column
// into the new column, then swap the new column into the old column.
// this struct contains everything we need to do that.
// we also need a backup column for safety - see usage of backupColumnName.
data class ColumnTypeChangePlan(
val realColumnName: String,
val tempColumnName: String,
val backupColumnName: String,
val originalType: StandardSQLTypeName,
val newType: StandardSQLTypeName,
)
val typeChangePlans: List<ColumnTypeChangePlan> =
columnsToChange.map { (name, originalType, newType) ->
// prefix with letter in case the SHA256 starts with a number
val nameHash = "a" + DigestUtils.sha256Hex(name)
val tempColumnName = "${nameHash}_airbyte_tmp"
val backupColumnName = "${nameHash}_airbyte_tmp_to_drop"
ColumnTypeChangePlan(
realColumnName = name,
tempColumnName = tempColumnName,
backupColumnName = backupColumnName,
originalType = originalType,
newType = newType,
)
}
val initialAlterations =
columnsToRemove.map { name -> """DROP COLUMN $name""" } +
columnsToAdd.map { (name, type) -> """ADD COLUMN $name $type""" } +
// in the initial statement, we just add the temporary column.
typeChangePlans.map { plan ->
"""ADD COLUMN ${plan.tempColumnName} ${plan.newType}"""
}
databaseHandler.executeWithRetries(
"""ALTER TABLE $tableId ${initialAlterations.joinToString(",")}"""
)
// now we execute the rest of the table alterations.
// these happen on a per-column basis, so that a failed UPDATE statement in one column
// doesn't block other schema changes from happening.
typeChangePlans.forEach {
(realColumnName, tempColumnName, backupColumnName, originalType, newType) ->
// first, update the temp column to contain the casted value.
val castStatement = getColumnCastStatement(realColumnName, originalType, newType)
try {
databaseHandler.executeWithRetries(
"""UPDATE $tableId SET $tempColumnName = $castStatement WHERE 1=1"""
)
} catch (e: Exception) {
val message =
"Error while updating schema for table ${tableName.toPrettyString()} (attempting to change column $realColumnName from $originalType to $newType). You should manually update the schema for this table."
logger.warn(e) { message }
// no rollback logic. On the next sync, we'll see the temp columns in columnsToDrop.
throw ConfigErrorException(message, e)
}
// then, swap the temp column to replace the original column.
// this is surprisingly nontrivial.
// bigquery doesn't support DDL in transactions,
// and also doesn't support having RENAME COLUMN and DROP COLUMN in the same
// ALTER TABLE statement.
// so this gives us the safest way to drop the old column:
// we atomically rename the old column to a holding location
// and rename the new column to replace it.
// Then, in a second ALTER TABLE, we drop the old column.
// this means that there's never a time when the table is completely missing
// the actual column.
// If we crash immediately after the RENAME COLUMNs, everything is fine:
// the next sync will see $backupColumnName as a column to drop,
// and we'll recover naturally.
databaseHandler.executeWithRetries(
"""
ALTER TABLE $tableId
RENAME COLUMN $realColumnName TO $backupColumnName,
RENAME COLUMN $tempColumnName TO $realColumnName
""".trimIndent(),
)
databaseHandler.executeWithRetries(
"""ALTER TABLE $tableId DROP COLUMN $backupColumnName""",
)
}
}
companion object {
@VisibleForTesting
fun clusteringMatches(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
existingTable: StandardTableDefinition,
): Boolean {
return (existingTable.clustering != null &&
containsAllIgnoreCase(
HashSet<String>(existingTable.clustering!!.fields),
BigqueryDirectLoadSqlGenerator.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
}
}
}

View File

@@ -0,0 +1,330 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables
import com.google.cloud.bigquery.StandardSQLTypeName
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.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.direct_load_table.DirectLoadSqlGenerator
import io.airbyte.integrations.destination.bigquery.spec.CdcDeletionMode
import java.util.ArrayList
import java.util.stream.Collectors
import org.apache.commons.lang3.StringUtils
class BigqueryDirectLoadSqlGenerator(
private val projectId: String?,
private val cdcDeletionMode: CdcDeletionMode,
) : DirectLoadSqlGenerator {
override fun createTable(
stream: DestinationStream,
tableName: TableName,
columnNameMapping: ColumnNameMapping,
replace: Boolean,
): Sql {
fun columnsAndTypes(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping
): String =
stream.schema
.asColumns()
.map { (fieldName, type) ->
val columnName = columnNameMapping[fieldName]!!
val typeName = toDialectType(type.type).name
"`$columnName` $typeName"
}
.joinToString(",\n")
val columnDeclarations = columnsAndTypes(stream, columnNameMapping)
val clusterConfig =
clusteringColumns(stream, columnNameMapping)
.stream()
.map { c: String? -> StringUtils.wrap(c, QUOTE) }
.collect(Collectors.joining(", "))
val forceCreateTable = if (replace) "OR REPLACE" else ""
val finalTableId = tableName.toPrettyString(QUOTE)
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()
)
}
override fun overwriteTable(sourceTableName: TableName, targetTableName: TableName): Sql {
throw NotImplementedError(
"This method is implemented using a native bigquery API call in BigqueryDirectLoadSqlTableOperations"
)
}
override fun copyTable(
columnNameMapping: ColumnNameMapping,
sourceTableName: TableName,
targetTableName: TableName
): Sql {
val columnNames = columnNameMapping.map { (_, actualName) -> actualName }.joinToString(",")
return Sql.of(
// TODO can we use CDK builtin stuff instead of hardcoding the airbyte meta columns?
"""
INSERT INTO `${targetTableName.namespace}`.`${targetTableName.name}`
(
_airbyte_raw_id,
_airbyte_extracted_at,
_airbyte_meta,
_airbyte_generation_id,
$columnNames
)
SELECT
_airbyte_raw_id,
_airbyte_extracted_at,
_airbyte_meta,
_airbyte_generation_id,
$columnNames
FROM `${sourceTableName.namespace}`.`${sourceTableName.name}`
""".trimIndent()
)
}
override fun upsertTable(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping,
sourceTableName: TableName,
targetTableName: TableName
): Sql {
val importType = stream.importType as Dedupe
val pkEquivalent =
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.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"`$columnName`,"
}
val newRecordColumnList: String =
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"new_record.`$columnName`,"
}
val selectSourceRecords = selectDedupedRecords(stream, sourceTableName, columnNameMapping)
val cursorComparison: String
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.
("""
(
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.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN) &&
cdcDeletionMode == CdcDeletionMode.HARD_DELETE
) {
// 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.schema.asColumns().keys.joinToString("\n") { fieldName ->
val column = columnNameMapping[fieldName]!!
"`$column` = new_record.`$column`,"
}
val targetTableId = targetTableName.toPrettyString(QUOTE)
return Sql.of(
"""
MERGE `$projectId`.$targetTableId target_table
USING (
$selectSourceRecords
) 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()
)
}
override fun dropTable(tableName: TableName): Sql {
val tableId = tableName.toPrettyString(QUOTE)
return Sql.of("""DROP TABLE IF EXISTS `$projectId`.$tableId;""")
}
/**
* A SQL SELECT statement that extracts records from the table and dedupes the records (since we
* only need the most-recent record to upsert).
*/
private fun selectDedupedRecords(
stream: DestinationStream,
sourceTableName: TableName,
columnNameMapping: ColumnNameMapping,
): String {
val columnList: String =
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
val columnName = columnNameMapping[fieldName]!!
"`$columnName`,"
}
val importType = stream.importType as Dedupe
// We need to dedup the records. Note the row_number() invocation in
// the SQL statement. We only take the most-recent raw record for each PK.
val pkList =
importType.primaryKey.joinToString(",") { fieldName ->
val columnName = columnNameMapping[fieldName.first()]!!
"`$columnName`"
}
val cursorOrderClause =
if (importType.cursor.isEmpty()) {
""
} else if (importType.cursor.size == 1) {
val columnName = columnNameMapping[importType.cursor.first()]!!
"`$columnName` DESC NULLS LAST,"
} else {
throw UnsupportedOperationException(
"Only top-level cursors are supported, got ${importType.cursor}"
)
}
return """
WITH records AS (
SELECT
$columnList
_airbyte_meta,
_airbyte_raw_id,
_airbyte_extracted_at,
_airbyte_generation_id
FROM `$projectId`.${sourceTableName.toPrettyString(QUOTE)}
), numbered_rows AS (
SELECT *, row_number() OVER (
PARTITION BY $pkList ORDER BY $cursorOrderClause `_airbyte_extracted_at` DESC
) AS row_number
FROM records
)
SELECT $columnList _airbyte_meta, _airbyte_raw_id, _airbyte_extracted_at, _airbyte_generation_id
FROM numbered_rows
WHERE row_number = 1
""".trimIndent()
}
companion object {
const val QUOTE: String = "`"
fun toDialectType(type: AirbyteType): StandardSQLTypeName =
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 -> StandardSQLTypeName.JSON
is UnionType ->
if (type.isLegacyUnion) {
toDialectType(type.chooseType())
} else {
StandardSQLTypeName.JSON
}
is UnknownType -> StandardSQLTypeName.JSON
}
fun clusteringColumns(
stream: DestinationStream,
columnNameMapping: ColumnNameMapping
): List<String> {
val clusterColumns: MutableList<String> = ArrayList()
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.importType as Dedupe).primaryKey.stream().limit(3).forEach {
pk: List<String> ->
clusterColumns.add(columnNameMapping[pk.first()]!!)
}
}
clusterColumns.add("_airbyte_extracted_at")
return clusterColumns
}
}
}

View File

@@ -0,0 +1,48 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables
import com.google.cloud.bigquery.BigQuery
import com.google.cloud.bigquery.CopyJobConfiguration
import com.google.cloud.bigquery.JobInfo
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
import io.airbyte.cdk.load.orchestration.db.TableName
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DefaultDirectLoadTableSqlOperations
import io.airbyte.cdk.load.orchestration.db.direct_load_table.DirectLoadTableSqlOperations
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
class BigqueryDirectLoadSqlTableOperations(
private val defaultOperations: DefaultDirectLoadTableSqlOperations,
private val bq: BigQuery,
) : DirectLoadTableSqlOperations by defaultOperations {
@SuppressFBWarnings("RV_RETURN_VALUE_IGNORED_NO_SIDE_EFFECT", "kotlin coroutines")
override suspend fun overwriteTable(sourceTableName: TableName, targetTableName: TableName) {
// manually delete the target table - otherwise we can't e.g. update the partitioning scheme
bq.getTable(targetTableName.toTableId())?.delete()
// Bigquery's SQL `ALTER TABLE RENAME TO` statement doesn't support moving tables
// across datasets.
// So we'll use a Copy job instead.
// (this is more efficient than just `insert into tgt select * from src`)
val sourceTableId = sourceTableName.toTableId()
val job =
bq.create(
JobInfo.of(
CopyJobConfiguration.newBuilder(
targetTableName.toTableId(),
sourceTableId,
)
// create the table if it doesn't yet exist
.setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED)
// overwrite the table if it already exists
.setWriteDisposition(JobInfo.WriteDisposition.WRITE_TRUNCATE)
.build()
)
)
BigQueryUtils.waitForJobFinish(job)
bq.getTable(sourceTableId).delete()
}
}

View File

@@ -2,7 +2,7 @@
* Copyright (c) 2025 Airbyte, Inc., all rights reserved. * Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/ */
package io.airbyte.integrations.destination.bigquery.write package io.airbyte.integrations.destination.bigquery.write.typing_deduping.legacy_raw_tables
import com.google.cloud.bigquery.BigQuery import com.google.cloud.bigquery.BigQuery
import com.google.cloud.bigquery.QueryJobConfiguration import com.google.cloud.bigquery.QueryJobConfiguration

View File

@@ -0,0 +1,109 @@
/*
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
*/
package io.airbyte.integrations.destination.bigquery.write.typing_deduping.legacy_raw_tables
import com.google.cloud.bigquery.BigQuery
import com.google.cloud.bigquery.QueryJobConfiguration
import com.google.cloud.bigquery.TableId
import io.airbyte.cdk.load.command.DestinationStream
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.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.TypingDedupingDatabaseInitialStatus
class BigqueryTypingDedupingDatabaseInitialStatusGatherer(private val bq: BigQuery) :
DatabaseInitialStatusGatherer<TypingDedupingDatabaseInitialStatus> {
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 null
val rawTableIdQuoted = """`${rawTableName.namespace}`.`${rawTableName.name}$suffix`"""
val unloadedRecordTimestamp =
bq.query(
QueryJobConfiguration.of(
"""
SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND)
FROM $rawTableIdQuoted
WHERE _airbyte_loaded_at IS NULL
""".trimIndent()
)
)
.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.of(
"""
SELECT MAX(_airbyte_extracted_at)
FROM $rawTableIdQuoted
""".trimIndent()
)
)
.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: TableCatalog,
): Map<DestinationStream, TypingDedupingDatabaseInitialStatus> {
return streams.mapValues { (stream, names) ->
val (tableNames, _) = names
// we're never actually doing anything with the final table
// so just return a hardcoded "safe" status
val finalTableStatus =
FinalTableInitialStatus(
isSchemaMismatch = false,
isEmpty = true,
finalTableGenerationId = stream.generationId,
)
val rawTableState = getInitialRawTableState(tableNames.rawTableName!!, "")
val tempRawTableState =
getInitialRawTableState(
tableNames.rawTableName!!,
TableNames.TMP_TABLE_SUFFIX,
)
TypingDedupingDatabaseInitialStatus(
finalTableStatus,
rawTableState,
tempRawTableState,
)
}
}
}

View File

@@ -10,6 +10,7 @@ import io.airbyte.cdk.load.util.serializeToString
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
import io.airbyte.integrations.destination.bigquery.spec.CdcDeletionMode
import java.io.IOException import java.io.IOException
import java.nio.file.Files import java.nio.file.Files
import java.nio.file.Path import java.nio.file.Path
@@ -34,13 +35,13 @@ object BigQueryDestinationTestUtils {
* generated by the test. * generated by the test.
* *
* @param configFile Path to the config file * @param configFile Path to the config file
* @param datasetId Dataset id to use in the test. Should be randomized per test case.
*/ */
@Throws(IOException::class) @Throws(IOException::class)
fun createConfig( fun createConfig(
configFile: String, configFile: String,
rawDatasetId: String? = null, rawDatasetId: String? = null,
disableTypingDeduping: Boolean? = null, disableTypingDeduping: Boolean? = null,
cdcDeletionMode: CdcDeletionMode? = null,
): String { ): String {
val tmpConfigAsString = Files.readString(Path.of(configFile)) val tmpConfigAsString = Files.readString(Path.of(configFile))
val config = Jsons.readTree(tmpConfigAsString) as ObjectNode val config = Jsons.readTree(tmpConfigAsString) as ObjectNode
@@ -49,6 +50,9 @@ object BigQueryDestinationTestUtils {
disableTypingDeduping?.let { disableTypingDeduping?.let {
config.put(BigQueryConsts.DISABLE_TYPE_DEDUPE, disableTypingDeduping) config.put(BigQueryConsts.DISABLE_TYPE_DEDUPE, disableTypingDeduping)
} }
cdcDeletionMode?.let {
config.put(BigQueryConsts.CDC_DELETION_MODE, cdcDeletionMode.cdcDeletionMode)
}
return config.serializeToString() return config.serializeToString()
} }

View File

@@ -30,8 +30,8 @@ import io.airbyte.cdk.load.util.Jsons
import io.airbyte.cdk.load.util.deserializeToNode import io.airbyte.cdk.load.util.deserializeToNode
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryFinalTableNameGenerator
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryRawTableNameGenerator
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Reason import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Reason
import io.github.oshai.kotlinlogging.KotlinLogging import io.github.oshai.kotlinlogging.KotlinLogging
@@ -51,7 +51,8 @@ object BigqueryRawTableDataDumper : DestinationDataDumper {
val (_, rawTableName) = val (_, rawTableName) =
BigqueryRawTableNameGenerator(config).getTableName(stream.mappedDescriptor) BigqueryRawTableNameGenerator(config).getTableName(stream.mappedDescriptor)
return bigquery.getTable(TableId.of(config.rawTableDataset, rawTableName))?.let { table -> return bigquery.getTable(TableId.of(config.internalTableDataset, rawTableName))?.let { table
->
val bigquerySchema = table.getDefinition<StandardTableDefinition>().schema!! val bigquerySchema = table.getDefinition<StandardTableDefinition>().schema!!
table.list(bigquerySchema).iterateAll().map { row -> table.list(bigquerySchema).iterateAll().map { row ->
OutputRecord( OutputRecord(
@@ -72,7 +73,7 @@ object BigqueryRawTableDataDumper : DestinationDataDumper {
} }
?: run { ?: run {
logger.warn { logger.warn {
"Raw table does not exist: ${config.rawTableDataset}.$rawTableName. Returning empty list." "Raw table does not exist: ${config.internalTableDataset}.$rawTableName. Returning empty list."
} }
emptyList() emptyList()
} }

View File

@@ -15,16 +15,21 @@ import kotlinx.coroutines.runBlocking
private val logger = KotlinLogging.logger {} private val logger = KotlinLogging.logger {}
// set a more aggressive retention policy.
// bigquery is _really_ slow at listing datasets/tables.
const val RETENTION_DAYS = 7L
object BigqueryDestinationCleaner : DestinationCleaner { object BigqueryDestinationCleaner : DestinationCleaner {
private val actualCleaner = private val actualCleaner =
BigqueryDestinationCleanerInstance( BigqueryInternalTableDatasetCleaner(
BigQueryDestinationTestUtils.standardInsertRawOverrideConfig BigQueryDestinationTestUtils.standardInsertRawOverrideConfig
) )
.compose( .compose(
BigqueryDestinationCleanerInstance( BigqueryInternalTableDatasetCleaner(
BigQueryDestinationTestUtils.standardInsertConfig BigQueryDestinationTestUtils.standardInsertConfig
) )
) )
.compose(BigqueryDatasetCleaner(BigQueryDestinationTestUtils.standardInsertConfig))
override fun cleanup() { override fun cleanup() {
// only run the cleaner sometimes - our nightlies will do this enough of the time // only run the cleaner sometimes - our nightlies will do this enough of the time
@@ -38,15 +43,13 @@ object BigqueryDestinationCleaner : DestinationCleaner {
} }
} }
class BigqueryDestinationCleanerInstance(private val configString: String) : DestinationCleaner { class BigqueryInternalTableDatasetCleaner(private val configString: String) : DestinationCleaner {
override fun cleanup() { override fun cleanup() {
val config = BigQueryDestinationTestUtils.parseConfig(configString) val config = BigQueryDestinationTestUtils.parseConfig(configString)
val bigquery = BigqueryBeansFactory().getBigqueryClient(config) val bigquery = BigqueryBeansFactory().getBigqueryClient(config)
runBlocking(Dispatchers.IO) { runBlocking(Dispatchers.IO) {
logger.info { "Cleaning up old raw tables in ${config.rawTableDataset}" } logger.info { "Cleaning up old raw tables in ${config.internalTableDataset}" }
var rawTables = bigquery.listTables(config.internalTableDataset)
var rawTables = bigquery.listTables(config.rawTableDataset)
// Page.iterateAll is _really_ slow, even if the interior function is `launch`-ed. // Page.iterateAll is _really_ slow, even if the interior function is `launch`-ed.
// Manually page through, and launch all the deletion work, so that we're always // Manually page through, and launch all the deletion work, so that we're always
// fetching new pages. // fetching new pages.
@@ -81,7 +84,15 @@ class BigqueryDestinationCleanerInstance(private val configString: String) : Des
break break
} }
} }
}
}
}
class BigqueryDatasetCleaner(private val configString: String) : DestinationCleaner {
override fun cleanup() {
val config = BigQueryDestinationTestUtils.parseConfig(configString)
val bigquery = BigqueryBeansFactory().getBigqueryClient(config)
runBlocking(Dispatchers.IO) {
logger.info { "Cleaning up old datasets in ${config.projectId}" } logger.info { "Cleaning up old datasets in ${config.projectId}" }
var datasets = bigquery.listDatasets(config.projectId) var datasets = bigquery.listDatasets(config.projectId)
while (true) { while (true) {
@@ -117,10 +128,4 @@ class BigqueryDestinationCleanerInstance(private val configString: String) : Des
} }
} }
} }
companion object {
// set a more aggressive retention policy.
// bigquery is _really_ slow at listing datasets/tables.
const val RETENTION_DAYS = 7L
}
} }

View File

@@ -4,20 +4,11 @@
package io.airbyte.integrations.destination.bigquery package io.airbyte.integrations.destination.bigquery
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.NamespaceMapper
import io.airbyte.cdk.load.data.ObjectType
import io.airbyte.cdk.load.message.InputRecord
import io.airbyte.cdk.load.test.util.DestinationDataDumper import io.airbyte.cdk.load.test.util.DestinationDataDumper
import io.airbyte.cdk.load.test.util.ExpectedRecordMapper import io.airbyte.cdk.load.test.util.ExpectedRecordMapper
import io.airbyte.cdk.load.test.util.OutputRecord
import io.airbyte.cdk.load.test.util.UncoercedExpectedRecordMapper import io.airbyte.cdk.load.test.util.UncoercedExpectedRecordMapper
import io.airbyte.cdk.load.test.util.destination_process.DockerizedDestinationFactory
import io.airbyte.cdk.load.toolkits.load.db.orchestration.ColumnNameModifyingMapper import io.airbyte.cdk.load.toolkits.load.db.orchestration.ColumnNameModifyingMapper
import io.airbyte.cdk.load.toolkits.load.db.orchestration.RootLevelTimestampsToUtcMapper import io.airbyte.cdk.load.toolkits.load.db.orchestration.RootLevelTimestampsToUtcMapper
import io.airbyte.cdk.load.toolkits.load.db.orchestration.TypingDedupingMetaChangeMapper
import io.airbyte.cdk.load.write.AllTypesBehavior import io.airbyte.cdk.load.write.AllTypesBehavior
import io.airbyte.cdk.load.write.BasicFunctionalityIntegrationTest import io.airbyte.cdk.load.write.BasicFunctionalityIntegrationTest
import io.airbyte.cdk.load.write.DedupBehavior import io.airbyte.cdk.load.write.DedupBehavior
@@ -30,11 +21,9 @@ import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils
import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils.RAW_DATASET_OVERRIDE import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils.RAW_DATASET_OVERRIDE
import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils.STANDARD_INSERT_CONFIG import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils.STANDARD_INSERT_CONFIG
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryColumnNameGenerator import io.airbyte.integrations.destination.bigquery.spec.CdcDeletionMode
import kotlin.test.assertEquals import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryColumnNameGenerator
import org.junit.jupiter.api.Assertions.assertTrue
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.junit.jupiter.api.assertAll
abstract class BigqueryWriteTest( abstract class BigqueryWriteTest(
configContents: String, configContents: String,
@@ -42,6 +31,7 @@ abstract class BigqueryWriteTest(
expectedRecordMapper: ExpectedRecordMapper, expectedRecordMapper: ExpectedRecordMapper,
isStreamSchemaRetroactive: Boolean, isStreamSchemaRetroactive: Boolean,
preserveUndeclaredFields: Boolean, preserveUndeclaredFields: Boolean,
commitDataIncrementallyToEmptyDestination: Boolean,
dedupBehavior: DedupBehavior?, dedupBehavior: DedupBehavior?,
nullEqualsUnset: Boolean, nullEqualsUnset: Boolean,
allTypesBehavior: AllTypesBehavior, allTypesBehavior: AllTypesBehavior,
@@ -61,6 +51,7 @@ abstract class BigqueryWriteTest(
preserveUndeclaredFields = preserveUndeclaredFields, preserveUndeclaredFields = preserveUndeclaredFields,
supportFileTransfer = false, supportFileTransfer = false,
commitDataIncrementally = false, commitDataIncrementally = false,
commitDataIncrementallyToEmptyDestination = commitDataIncrementallyToEmptyDestination,
allTypesBehavior = allTypesBehavior, allTypesBehavior = allTypesBehavior,
nullEqualsUnset = nullEqualsUnset, nullEqualsUnset = nullEqualsUnset,
configUpdater = BigqueryConfigUpdater, configUpdater = BigqueryConfigUpdater,
@@ -76,22 +67,41 @@ abstract class BigqueryRawTablesWriteTest(
UncoercedExpectedRecordMapper, UncoercedExpectedRecordMapper,
isStreamSchemaRetroactive = false, isStreamSchemaRetroactive = false,
preserveUndeclaredFields = true, preserveUndeclaredFields = true,
commitDataIncrementallyToEmptyDestination = false,
dedupBehavior = null, dedupBehavior = null,
nullEqualsUnset = false, nullEqualsUnset = false,
Untyped, Untyped,
) )
abstract class BigqueryTDWriteTest(configContents: String) : abstract class BigqueryDirectLoadWriteTest(
configContents: String,
cdcDeletionMode: CdcDeletionMode,
) :
BigqueryWriteTest( BigqueryWriteTest(
configContents = configContents, configContents = configContents,
BigqueryFinalTableDataDumper, BigqueryFinalTableDataDumper,
ColumnNameModifyingMapper(BigqueryColumnNameGenerator()) ColumnNameModifyingMapper(BigqueryColumnNameGenerator())
.compose(TimeWithTimezoneMapper)
.compose(RootLevelTimestampsToUtcMapper) .compose(RootLevelTimestampsToUtcMapper)
.compose(TypingDedupingMetaChangeMapper)
.compose(IntegralNumberRecordMapper), .compose(IntegralNumberRecordMapper),
isStreamSchemaRetroactive = true, isStreamSchemaRetroactive = true,
preserveUndeclaredFields = false, preserveUndeclaredFields = false,
dedupBehavior = DedupBehavior(), commitDataIncrementallyToEmptyDestination = true,
dedupBehavior =
DedupBehavior(
cdcDeletionMode =
when (cdcDeletionMode) {
// medium confidence: the CDK might eventually add other deletion modes,
// which this destination won't immediately support,
// so we should have separate enums.
// otherwise the new enum values would show up in the spec, which we don't
// want.
CdcDeletionMode.HARD_DELETE ->
io.airbyte.cdk.load.write.DedupBehavior.CdcDeletionMode.HARD_DELETE
CdcDeletionMode.SOFT_DELETE ->
io.airbyte.cdk.load.write.DedupBehavior.CdcDeletionMode.SOFT_DELETE
}
),
nullEqualsUnset = true, nullEqualsUnset = true,
StronglyTyped( StronglyTyped(
convertAllValuesToString = true, convertAllValuesToString = true,
@@ -99,247 +109,66 @@ abstract class BigqueryTDWriteTest(configContents: String) :
nestedFloatLosesPrecision = true, nestedFloatLosesPrecision = true,
integerCanBeLarge = false, integerCanBeLarge = false,
numberCanBeLarge = false, numberCanBeLarge = false,
timeWithTimezoneBehavior = SimpleValueBehavior.PASS_THROUGH, numberIsFixedPointPrecision38Scale9 = true,
timeWithTimezoneBehavior = SimpleValueBehavior.STRONGLY_TYPE,
), ),
) { )
private val oldCdkDestinationFactory =
DockerizedDestinationFactory("airbyte/destination-bigquery", "2.10.2")
@Test class StandardInsertRawOverrideRawTables :
open fun testAppendCdkMigration() {
val stream =
DestinationStream(
unmappedNamespace = randomizedNamespace,
unmappedName = "test_stream",
Append,
ObjectType(linkedMapOf("id" to intType)),
generationId = 0,
minimumGenerationId = 0,
syncId = 42,
namespaceMapper = NamespaceMapper()
)
// Run a sync on the old CDK
runSync(
updatedConfig,
stream,
listOf(
InputRecord(
stream,
data = """{"id": 1234}""",
emittedAtMs = 1234,
),
),
destinationProcessFactory = oldCdkDestinationFactory,
)
// Grab the loaded_at value from this sync
val firstSyncLoadedAt =
BigqueryRawTableDataDumper.dumpRecords(parsedConfig, stream).first().loadedAt!!
// Run a sync with the current destination
runSync(
updatedConfig,
stream,
listOf(
InputRecord(
stream,
data = """{"id": 1234}""",
emittedAtMs = 5678,
),
),
)
val secondSyncLoadedAt =
BigqueryRawTableDataDumper.dumpRecords(parsedConfig, stream)
.map { it.loadedAt!! }
.toSet()
// verify that we didn't execute a soft reset
assertAll(
{
assertEquals(
2,
secondSyncLoadedAt.size,
"Expected two unique values for loaded_at after two syncs. If there is only 1 value, then we likely executed a soft reset.",
)
},
{
assertTrue(
secondSyncLoadedAt.contains(firstSyncLoadedAt),
"Expected the first sync's loaded_at value to exist after the second sync. If this is not true, then we likely executed a soft reset.",
)
},
)
dumpAndDiffRecords(
parsedConfig,
listOf(
OutputRecord(
extractedAt = 1234,
generationId = 0,
data = mapOf("id" to 1234),
airbyteMeta = OutputRecord.Meta(syncId = 42, changes = emptyList()),
),
OutputRecord(
extractedAt = 5678,
generationId = 0,
data = mapOf("id" to 1234),
airbyteMeta = OutputRecord.Meta(syncId = 42, changes = emptyList()),
),
),
stream,
listOf(listOf("id")),
cursor = null,
)
}
@Test
open fun testDedupCdkMigration() {
val stream =
DestinationStream(
unmappedNamespace = randomizedNamespace,
unmappedName = "test_stream",
Dedupe(primaryKey = listOf(listOf("id")), cursor = emptyList()),
ObjectType(linkedMapOf("id" to intType)),
generationId = 0,
minimumGenerationId = 0,
syncId = 42,
namespaceMapper = NamespaceMapper(),
)
// Run a sync on the old CDK
runSync(
updatedConfig,
stream,
listOf(
InputRecord(
stream,
data = """{"id": 1234}""",
emittedAtMs = 1234,
),
),
destinationProcessFactory = oldCdkDestinationFactory,
)
// Grab the loaded_at value from this sync
val firstSyncLoadedAt =
BigqueryRawTableDataDumper.dumpRecords(parsedConfig, stream).first().loadedAt!!
// Run a sync with the current destination
runSync(
updatedConfig,
stream,
listOf(
InputRecord(
stream = stream,
data = """{"id": 1234}""",
emittedAtMs = 5678,
),
),
)
val secondSyncLoadedAt =
BigqueryRawTableDataDumper.dumpRecords(parsedConfig, stream)
.map { it.loadedAt!! }
.toSet()
// verify that we didn't execute a soft reset
assertAll(
{
assertEquals(
2,
secondSyncLoadedAt.size,
"Expected two unique values for loaded_at after two syncs. If there is only 1 value, then we likely executed a soft reset.",
)
},
{
assertTrue(
secondSyncLoadedAt.contains(firstSyncLoadedAt),
"Expected the first sync's loaded_at value to exist after the second sync. If this is not true, then we likely executed a soft reset.",
)
},
)
dumpAndDiffRecords(
parsedConfig,
listOf(
OutputRecord(
extractedAt = 5678,
generationId = 0,
data = mapOf("id" to 1234),
airbyteMeta = OutputRecord.Meta(syncId = 42, changes = emptyList()),
),
),
stream,
listOf(listOf("id")),
cursor = null,
)
}
}
class StandardInsertRawOverrideDisableTd :
BigqueryRawTablesWriteTest( BigqueryRawTablesWriteTest(
BigQueryDestinationTestUtils.createConfig( BigQueryDestinationTestUtils.createConfig(
configFile = STANDARD_INSERT_CONFIG, configFile = STANDARD_INSERT_CONFIG,
rawDatasetId = RAW_DATASET_OVERRIDE, rawDatasetId = RAW_DATASET_OVERRIDE,
disableTypingDeduping = true, disableTypingDeduping = true,
), ),
) { )
@Test
override fun testBasicWrite() {
super.testBasicWrite()
}
@Test
override fun testAppendSchemaEvolution() {
super.testAppendSchemaEvolution()
}
}
class StandardInsertRawOverride : class StandardInsertRawOverride :
BigqueryTDWriteTest(BigQueryDestinationTestUtils.standardInsertRawOverrideConfig) { BigqueryDirectLoadWriteTest(
@Test BigQueryDestinationTestUtils.standardInsertRawOverrideConfig,
override fun testBasicWrite() { CdcDeletionMode.HARD_DELETE,
super.testBasicWrite() )
}
@Test
override fun testFunkyCharacters() {
super.testFunkyCharacters()
}
}
class StandardInsert : BigqueryTDWriteTest(BigQueryDestinationTestUtils.standardInsertConfig) { class StandardInsert :
BigqueryDirectLoadWriteTest(
BigQueryDestinationTestUtils.standardInsertConfig,
CdcDeletionMode.HARD_DELETE,
)
class StandardInsertCdcSoftDeletes :
BigqueryDirectLoadWriteTest(
BigQueryDestinationTestUtils.createConfig(
configFile = STANDARD_INSERT_CONFIG,
cdcDeletionMode = CdcDeletionMode.SOFT_DELETE,
),
CdcDeletionMode.SOFT_DELETE
) {
@Test @Test
override fun testDedup() { override fun testDedup() {
super.testDedup() super.testDedup()
} }
} }
class GcsRawOverrideDisableTd : class GcsRawOverrideRawTables :
BigqueryRawTablesWriteTest( BigqueryRawTablesWriteTest(
BigQueryDestinationTestUtils.createConfig( BigQueryDestinationTestUtils.createConfig(
configFile = GCS_STAGING_CONFIG, configFile = GCS_STAGING_CONFIG,
rawDatasetId = RAW_DATASET_OVERRIDE, rawDatasetId = RAW_DATASET_OVERRIDE,
disableTypingDeduping = true, disableTypingDeduping = true,
), ),
) { )
@Test
override fun testBasicWrite() {
super.testBasicWrite()
}
}
class GcsRawOverride : class GcsRawOverride :
BigqueryTDWriteTest( BigqueryDirectLoadWriteTest(
BigQueryDestinationTestUtils.createConfig( BigQueryDestinationTestUtils.createConfig(
configFile = GCS_STAGING_CONFIG, configFile = GCS_STAGING_CONFIG,
rawDatasetId = RAW_DATASET_OVERRIDE, rawDatasetId = RAW_DATASET_OVERRIDE,
), ),
) { CdcDeletionMode.HARD_DELETE,
@Test )
override fun testBasicWrite() {
super.testBasicWrite()
}
}
class Gcs : class Gcs :
BigqueryTDWriteTest( BigqueryDirectLoadWriteTest(
BigQueryDestinationTestUtils.createConfig(configFile = GCS_STAGING_CONFIG) BigQueryDestinationTestUtils.createConfig(configFile = GCS_STAGING_CONFIG),
) { CdcDeletionMode.HARD_DELETE,
@Test )
override fun testBasicWrite() {
super.testBasicWrite()
}
}

View File

@@ -10,8 +10,11 @@ import io.airbyte.cdk.load.data.ArrayValue
import io.airbyte.cdk.load.data.IntegerValue import io.airbyte.cdk.load.data.IntegerValue
import io.airbyte.cdk.load.data.NumberValue import io.airbyte.cdk.load.data.NumberValue
import io.airbyte.cdk.load.data.ObjectValue import io.airbyte.cdk.load.data.ObjectValue
import io.airbyte.cdk.load.data.StringValue
import io.airbyte.cdk.load.data.TimeWithTimezoneValue
import io.airbyte.cdk.load.test.util.ExpectedRecordMapper import io.airbyte.cdk.load.test.util.ExpectedRecordMapper
import io.airbyte.cdk.load.test.util.OutputRecord import io.airbyte.cdk.load.test.util.OutputRecord
import java.time.format.DateTimeFormatter
/** /**
* In nested JSON fields, bigquery converts integral numbers to integers. For example, if you try to * In nested JSON fields, bigquery converts integral numbers to integers. For example, if you try to
@@ -56,3 +59,23 @@ object IntegralNumberRecordMapper : ExpectedRecordMapper {
else -> value else -> value
} }
} }
/**
* Bigquery doesn't have a timetz data type, so we use a STRING column. Which means that we need to
* map the expected values to string.
*/
object TimeWithTimezoneMapper : ExpectedRecordMapper {
override fun mapRecord(expectedRecord: OutputRecord, schema: AirbyteType): OutputRecord {
val mappedData =
ObjectValue(
expectedRecord.data.values.mapValuesTo(linkedMapOf()) { (_, value) ->
when (value) {
is TimeWithTimezoneValue ->
StringValue(value.value.format((DateTimeFormatter.ISO_OFFSET_TIME)))
else -> value
}
}
)
return expectedRecord.copy(data = mappedData)
}
}

View File

@@ -129,35 +129,39 @@
"airbyte_secret" : true, "airbyte_secret" : true,
"always_show" : true "always_show" : true
}, },
"transformation_priority" : { "cdc_deletion_mode" : {
"type" : "string", "type" : "string",
"default" : "interactive", "default" : "Hard delete",
"enum" : [ "interactive", "batch" ], "enum" : [ "Hard delete", "Soft delete" ],
"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 <a href=\"https://cloud.google.com/bigquery/docs/running-queries#queries\">here</a>. 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 dont count towards your concurrent rate limit. Read more about batch queries <a href=\"https://cloud.google.com/bigquery/docs/running-queries#batch\">here</a>. The default \"interactive\" value is used if not set explicitly.", "description" : "Whether to execute CDC deletions as hard deletes (i.e. propagate source deletions to the destination), or soft deletes (i.e. leave a tombstone record in the destination). Defaults to hard deletes.",
"title" : "Transformation Query Run Type", "title" : "CDC deletion mode",
"group" : "advanced", "group" : "sync_behavior",
"order" : 5 "order" : 5,
}, "always_show" : true
"raw_data_dataset" : {
"type" : "string",
"description" : "The dataset to write raw tables into (default: airbyte_internal)",
"title" : "Raw Table Dataset Name",
"group" : "advanced",
"order" : 7
}, },
"disable_type_dedupe" : { "disable_type_dedupe" : {
"type" : "boolean", "type" : "boolean",
"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", "description" : "Write the legacy \"raw tables\" format, to enable backwards compatibility with older versions of this connector.",
"title" : "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)", "title" : "Legacy raw tables",
"group" : "advanced", "group" : "advanced",
"order" : 8, "order" : 7,
"default" : false "default" : false
},
"raw_data_dataset" : {
"type" : "string",
"description" : "Airbyte will use this dataset for various internal tables. In legacy raw tables mode, the raw tables will be stored in this dataset. Defaults to \"airbyte_internal\".",
"title" : "Airbyte Internal Table Dataset Name",
"group" : "advanced",
"order" : 8
} }
}, },
"required" : [ "project_id", "dataset_location", "dataset_id" ], "required" : [ "project_id", "dataset_location", "dataset_id" ],
"groups" : [ { "groups" : [ {
"id" : "connection", "id" : "connection",
"title" : "Connection" "title" : "Connection"
}, {
"id" : "sync_behavior",
"title" : "Sync Behavior"
}, { }, {
"id" : "advanced", "id" : "advanced",
"title" : "Advanced" "title" : "Advanced"

View File

@@ -129,35 +129,39 @@
"airbyte_secret" : true, "airbyte_secret" : true,
"always_show" : true "always_show" : true
}, },
"transformation_priority" : { "cdc_deletion_mode" : {
"type" : "string", "type" : "string",
"default" : "interactive", "default" : "Hard delete",
"enum" : [ "interactive", "batch" ], "enum" : [ "Hard delete", "Soft delete" ],
"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 <a href=\"https://cloud.google.com/bigquery/docs/running-queries#queries\">here</a>. 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 dont count towards your concurrent rate limit. Read more about batch queries <a href=\"https://cloud.google.com/bigquery/docs/running-queries#batch\">here</a>. The default \"interactive\" value is used if not set explicitly.", "description" : "Whether to execute CDC deletions as hard deletes (i.e. propagate source deletions to the destination), or soft deletes (i.e. leave a tombstone record in the destination). Defaults to hard deletes.",
"title" : "Transformation Query Run Type", "title" : "CDC deletion mode",
"group" : "advanced", "group" : "sync_behavior",
"order" : 5 "order" : 5,
}, "always_show" : true
"raw_data_dataset" : {
"type" : "string",
"description" : "The dataset to write raw tables into (default: airbyte_internal)",
"title" : "Raw Table Dataset Name",
"group" : "advanced",
"order" : 7
}, },
"disable_type_dedupe" : { "disable_type_dedupe" : {
"type" : "boolean", "type" : "boolean",
"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", "description" : "Write the legacy \"raw tables\" format, to enable backwards compatibility with older versions of this connector.",
"title" : "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)", "title" : "Legacy raw tables",
"group" : "advanced", "group" : "advanced",
"order" : 8, "order" : 7,
"default" : false "default" : false
},
"raw_data_dataset" : {
"type" : "string",
"description" : "Airbyte will use this dataset for various internal tables. In legacy raw tables mode, the raw tables will be stored in this dataset. Defaults to \"airbyte_internal\".",
"title" : "Airbyte Internal Table Dataset Name",
"group" : "advanced",
"order" : 8
} }
}, },
"required" : [ "project_id", "dataset_location", "dataset_id" ], "required" : [ "project_id", "dataset_location", "dataset_id" ],
"groups" : [ { "groups" : [ {
"id" : "connection", "id" : "connection",
"title" : "Connection" "title" : "Connection"
}, {
"id" : "sync_behavior",
"title" : "Sync Behavior"
}, { }, {
"id" : "advanced", "id" : "advanced",
"title" : "Advanced" "title" : "Advanced"

View File

@@ -1,7 +1,7 @@
/* /*
* Copyright (c) 2023 Airbyte, Inc., all rights reserved. * Copyright (c) 2023 Airbyte, Inc., all rights reserved.
*/ */
package io.airbyte.integrations.destination.bigquery.typing_deduping package io.airbyte.integrations.destination.bigquery.typing_deduping.direct_load_tables
import com.google.cloud.bigquery.Clustering import com.google.cloud.bigquery.Clustering
import com.google.cloud.bigquery.Field import com.google.cloud.bigquery.Field
@@ -22,15 +22,19 @@ import io.airbyte.cdk.load.data.ObjectType
import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema import io.airbyte.cdk.load.data.ObjectTypeWithoutSchema
import io.airbyte.cdk.load.data.UnionType import io.airbyte.cdk.load.data.UnionType
import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.toDialectType import io.airbyte.cdk.load.orchestration.db.DefaultTempTableNameGenerator
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDatabaseInitialStatusGatherer.Companion.clusteringMatches import io.airbyte.cdk.load.orchestration.db.direct_load_table.ColumnAdd
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDatabaseInitialStatusGatherer.Companion.partitioningMatches import io.airbyte.cdk.load.orchestration.db.direct_load_table.ColumnChange
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadNativeTableOperations
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadNativeTableOperations.Companion.clusteringMatches
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadNativeTableOperations.Companion.partitioningMatches
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.direct_load_tables.BigqueryDirectLoadSqlGenerator.Companion.toDialectType
import org.junit.jupiter.api.Assertions import org.junit.jupiter.api.Assertions
import org.junit.jupiter.api.Test import org.junit.jupiter.api.Test
import org.mockito.Mockito import org.mockito.Mockito
import org.mockito.Mockito.RETURNS_DEEP_STUBS import org.mockito.Mockito.RETURNS_DEEP_STUBS
class BigqueryDestinationHandlerTest { class BigqueryDirectLoadNativeTableOperationsTest {
@Test @Test
fun testToDialectType() { fun testToDialectType() {
val s = ObjectType(linkedMapOf()) val s = ObjectType(linkedMapOf())
@@ -70,12 +74,30 @@ class BigqueryDestinationHandlerTest {
Mockito.`when`(existingTable.schema!!.fields) Mockito.`when`(existingTable.schema!!.fields)
.thenReturn(FieldList.of(Field.of("a2", StandardSQLTypeName.INT64))) .thenReturn(FieldList.of(Field.of("a2", StandardSQLTypeName.INT64)))
val alterTableReport = val alterTableReport =
BigqueryDatabaseInitialStatusGatherer(Mockito.mock()) BigqueryDirectLoadNativeTableOperations(
Mockito.mock(),
Mockito.mock(),
Mockito.mock(),
projectId = "unused",
tempTableNameGenerator = DefaultTempTableNameGenerator("unused"),
)
.buildAlterTableReport(stream, columnNameMapping, existingTable) .buildAlterTableReport(stream, columnNameMapping, existingTable)
Assertions.assertAll( Assertions.assertAll(
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToAdd) }, {
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToRemove) }, Assertions.assertEquals(
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToChangeType) }, emptyList<Pair<String, StandardSQLTypeName>>(),
alterTableReport.columnsToAdd
)
},
{ Assertions.assertEquals(emptyList<String>(), alterTableReport.columnsToRemove) },
{
Assertions.assertEquals(
emptyList<ColumnChange<StandardSQLTypeName>>(),
alterTableReport.columnsToChangeType
)
},
// NB: column names in AlterTableReport are all _after_ destination name transform
{ Assertions.assertEquals(listOf("a2"), alterTableReport.columnsToRetain) },
) )
} }
@@ -109,12 +131,41 @@ class BigqueryDestinationHandlerTest {
) )
) )
val alterTableReport = val alterTableReport =
BigqueryDatabaseInitialStatusGatherer(Mockito.mock()) BigqueryDirectLoadNativeTableOperations(
Mockito.mock(),
Mockito.mock(),
Mockito.mock(),
projectId = "unused",
tempTableNameGenerator = DefaultTempTableNameGenerator("unused"),
)
.buildAlterTableReport(stream, columnNameMapping, existingTable) .buildAlterTableReport(stream, columnNameMapping, existingTable)
// NB: column names in AlterTableReport are all _after_ destination name transform
Assertions.assertAll( Assertions.assertAll(
{ Assertions.assertEquals(setOf("c2"), alterTableReport.columnsToAdd) }, {
{ Assertions.assertEquals(setOf("b2"), alterTableReport.columnsToRemove) }, Assertions.assertEquals(
{ Assertions.assertEquals(setOf("a2"), alterTableReport.columnsToChangeType) }, listOf(ColumnAdd("c2", StandardSQLTypeName.INT64)),
alterTableReport.columnsToAdd
)
},
{ Assertions.assertEquals(listOf("b2"), alterTableReport.columnsToRemove) },
{
Assertions.assertEquals(
listOf(
ColumnChange(
name = "a2",
originalType = StandardSQLTypeName.STRING,
newType = StandardSQLTypeName.INT64,
)
),
alterTableReport.columnsToChangeType,
)
},
{
Assertions.assertEquals(
emptyList<ColumnChange<StandardSQLTypeName>>(),
alterTableReport.columnsToRetain
)
}
) )
} }

View File

@@ -191,6 +191,7 @@ class AirbyteBulkConnectorPlugin implements Plugin<Project> {
testLogging() { testLogging() {
events 'skipped', 'started', 'passed', 'failed' events 'skipped', 'started', 'passed', 'failed'
exceptionFormat 'full' exceptionFormat 'full'
showStandardStreams = true
} }
// Always re-run integration tests no matter what. // Always re-run integration tests no matter what.

View File

@@ -1,5 +1,17 @@
# BigQuery Migration Guide # BigQuery Migration Guide
## Upgrading to 3.0.0
This version upgrades Destination BigQuery to the [Direct-Load](/platform/using-airbyte/core-concepts/direct-load-tables) paradigm, which improves performance and reduces warehouse spend. If you have unusual requirements around record visibility or schema evolution, read that document for more information about how direct-load differs from Typing and Deduping.
This version also adds an option to enable CDC deletions as soft-deletes.
If you do not interact with the raw tables, you can safely upgrade. There is no breakage for this usecase.
If you _only_ interact with the raw tables, make sure that you have the `Disable Final Tables` option enabled before upgrading. This will automatically enable the `Legacy raw tables` option after upgrading.
If you interact with both the raw _and_ final tables, this usecase will no longer be directly supported. You must create two connectors (one with `Disable Final Tables` enabled, and one with it disabled) and run two connections in parallel.
## Upgrading to 2.0.0 ## Upgrading to 2.0.0
This version introduces [Destinations V2](/release_notes/upgrading_to_destinations_v2/#what-is-destinations-v2), which provides better error handling, incremental delivery of data for large syncs, and improved final table structures. To review the breaking changes, and how to upgrade, see [here](/release_notes/upgrading_to_destinations_v2/#quick-start-to-upgrading). These changes will likely require updates to downstream dbt / SQL models, which we walk through [here](/release_notes/upgrading_to_destinations_v2/#updating-downstream-transformations). Selecting `Upgrade` will upgrade **all** connections using this destination at their next sync. You can manually sync existing connections prior to the next scheduled sync to start the upgrade early. This version introduces [Destinations V2](/release_notes/upgrading_to_destinations_v2/#what-is-destinations-v2), which provides better error handling, incremental delivery of data for large syncs, and improved final table structures. To review the breaking changes, and how to upgrade, see [here](/release_notes/upgrading_to_destinations_v2/#quick-start-to-upgrading). These changes will likely require updates to downstream dbt / SQL models, which we walk through [here](/release_notes/upgrading_to_destinations_v2/#updating-downstream-transformations). Selecting `Upgrade` will upgrade **all** connections using this destination at their next sync. You can manually sync existing connections prior to the next scheduled sync to start the upgrade early.

View File

@@ -115,28 +115,6 @@ The BigQuery destination connector supports the following
## Output schema ## Output schema
Airbyte outputs each stream into its own raw table in `airbyte_internal` dataset by default (can be
overriden by user) and a final table with Typed columns. Contents in raw table are _NOT_
deduplicated.
### Raw Table schema
The raw table contains these fields:
- `_airbyte_raw_id`
- `_airbyte_generation_id`
- `_airbyte_extracted_at`
- `_airbyte_loaded_at`
- `_airbyte_meta`
- `_airbyte_data`
`_airbyte_data` is a JSON blob with the event data. See [here](/platform/understanding-airbyte/airbyte-metadata-fields)
for more information about the other fields.
**Note:** Although the contents of the `_airbyte_data` are fairly stable, schema of the raw table
could be subject to change in future versions.
### Final Table schema
The final table contains these fields, in addition to the columns declared in your stream schema: The final table contains these fields, in addition to the columns declared in your stream schema:
- `airbyte_raw_id` - `airbyte_raw_id`
- `_airbyte_generation_id` - `_airbyte_generation_id`
@@ -153,6 +131,25 @@ partitioning column are used to prune the partitions and reduce the query cost.
**Require partition filter** is not enabled by Airbyte, but you may toggle it by updating the **Require partition filter** is not enabled by Airbyte, but you may toggle it by updating the
produced tables.) produced tables.)
### Legacy Raw Tables schema
If you enable the `Legacy raw tables` option, the connector will write tables in this format.
Airbyte outputs each stream into its own raw table in `airbyte_internal` dataset by default (you can
override this via the `Airbyte Internal Table Dataset Name` option). Contents in the raw table are
_NOT_ deduplicated.
The raw table contains these fields:
- `_airbyte_raw_id`
- `_airbyte_generation_id`
- `_airbyte_extracted_at`
- `_airbyte_loaded_at`
- `_airbyte_meta`
- `_airbyte_data`
`_airbyte_data` is a JSON blob with the record's data. See [here](/platform/understanding-airbyte/airbyte-metadata-fields)
for more information about the other fields.
## BigQuery Naming Conventions ## BigQuery Naming Conventions
Follow Follow
@@ -213,6 +210,7 @@ tutorials:
| Version | Date | Pull Request | Subject | | Version | Date | Pull Request | Subject |
|:------------|:-----------|:-----------------------------------------------------------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------| |:------------|:-----------|:-----------------------------------------------------------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
| 3.0.0 | 2025-06-25 | [59752](https://github.com/airbytehq/airbyte/pull/59752) | Upgrade to direct-load tables; add option for soft CDC deletes. |
| 2.12.4 | 2025-06-24 | [62045](https://github.com/airbytehq/airbyte/pull/62045) | Promoting release candidate 2.12.4-rc.6 to a main version. | | 2.12.4 | 2025-06-24 | [62045](https://github.com/airbytehq/airbyte/pull/62045) | Promoting release candidate 2.12.4-rc.6 to a main version. |
| 2.12.4-rc.6 | 2025-06-24 | [62041](https://github.com/airbytehq/airbyte/pull/62041) | 2.12.4 RC 6 More retries is better retries | | 2.12.4-rc.6 | 2025-06-24 | [62041](https://github.com/airbytehq/airbyte/pull/62041) | 2.12.4 RC 6 More retries is better retries |
| 2.12.4-rc.5 | 2025-06-23 | [62016](https://github.com/airbytehq/airbyte/pull/62016) | 2.12.4 RC 5 Handle records up to 20MiB | | 2.12.4-rc.5 | 2025-06-23 | [62016](https://github.com/airbytehq/airbyte/pull/62016) | 2.12.4 RC 5 Handle records up to 20MiB |