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:
@@ -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())
|
||||
}
|
||||
}
|
||||
@@ -6,7 +6,7 @@ data:
|
||||
connectorSubtype: database
|
||||
connectorType: destination
|
||||
definitionId: 22f6c74f-5699-40ff-833c-4a879ea40133
|
||||
dockerImageTag: 2.12.4
|
||||
dockerImageTag: 3.0.0
|
||||
dockerRepository: airbyte/destination-bigquery
|
||||
documentationUrl: https://docs.airbyte.com/integrations/destinations/bigquery
|
||||
githubIssueLabel: destination-bigquery
|
||||
@@ -26,6 +26,9 @@ data:
|
||||
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"
|
||||
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:
|
||||
enableProgressiveRollout: false
|
||||
resourceRequirements:
|
||||
|
||||
@@ -4,12 +4,10 @@
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
object BigQueryConsts {
|
||||
const val MiB: Int = 1024 * 1024
|
||||
const val CONFIG_DATASET_ID: String = "dataset_id"
|
||||
const val CONFIG_PROJECT_ID: String = "project_id"
|
||||
const val CONFIG_DATASET_LOCATION: String = "dataset_location"
|
||||
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 METHOD: String = "method"
|
||||
@@ -19,9 +17,9 @@ object BigQueryConsts {
|
||||
const val GCS_BUCKET_REGION: String = "gcs_bucket_region"
|
||||
const val CREDENTIAL: String = "credential"
|
||||
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 RAW_DATA_DATASET = "raw_data_dataset"
|
||||
const val CDC_DELETION_MODE: String = "cdc_deletion_mode"
|
||||
const val NAMESPACE_PREFIX: String = "n"
|
||||
const val NULL_MARKER: String = "\\N"
|
||||
}
|
||||
|
||||
@@ -11,22 +11,31 @@ import com.google.cloud.bigquery.BigQueryOptions
|
||||
import io.airbyte.cdk.load.check.DestinationCheckerSync
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
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.TypingDedupingExecutionConfig
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingFinalTableOperations
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingWriter
|
||||
import io.airbyte.cdk.load.state.SyncManager
|
||||
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.WriteOperation
|
||||
import io.airbyte.integrations.destination.bigquery.check.BigqueryCheckCleaner
|
||||
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.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.micronaut.context.annotation.Factory
|
||||
import io.micronaut.context.annotation.Requires
|
||||
@@ -70,21 +79,69 @@ class BigqueryBeansFactory {
|
||||
bigquery: BigQuery,
|
||||
config: BigqueryConfiguration,
|
||||
names: TableCatalog,
|
||||
streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
): TypingDedupingWriter {
|
||||
// micronaut will only instantiate a single instance of StreamStateStore,
|
||||
// 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)
|
||||
if (config.legacyRawTablesOnly) {
|
||||
// force smart cast
|
||||
@Suppress("UNCHECKED_CAST")
|
||||
streamStateStore as StreamStateStore<TypingDedupingExecutionConfig>
|
||||
return TypingDedupingWriter(
|
||||
names,
|
||||
BigqueryDatabaseInitialStatusGatherer(bigquery),
|
||||
BigqueryTypingDedupingDatabaseInitialStatusGatherer(bigquery),
|
||||
destinationHandler,
|
||||
BigqueryRawTableOperations(bigquery),
|
||||
TypingDedupingFinalTableOperations(
|
||||
BigQuerySqlGenerator(config.projectId, config.datasetLocation.region),
|
||||
NoopTypingDedupingSqlGenerator,
|
||||
destinationHandler,
|
||||
),
|
||||
disableTypeDedupe = config.disableTypingDeduping,
|
||||
streamStateStore,
|
||||
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
|
||||
|
||||
@@ -8,9 +8,9 @@ import io.airbyte.cdk.load.check.CheckCleaner
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.integrations.destination.bigquery.BigqueryBeansFactory
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.toTableId
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryRawTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
|
||||
|
||||
class BigqueryCheckCleaner : CheckCleaner<BigqueryConfiguration> {
|
||||
override fun cleanup(config: BigqueryConfiguration, stream: DestinationStream) {
|
||||
|
||||
@@ -3,55 +3,133 @@
|
||||
*/
|
||||
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.QueryParameterValue
|
||||
import com.google.cloud.bigquery.Schema
|
||||
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.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.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.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.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
|
||||
|
||||
/**
|
||||
* The class formats incoming JsonSchema and AirbyteRecord in order to be inline with a
|
||||
* corresponding uploader.
|
||||
*/
|
||||
class BigQueryRecordFormatter {
|
||||
class BigQueryRecordFormatter(
|
||||
private val columnNameMapping: ColumnNameMapping,
|
||||
private val legacyRawTablesOnly: Boolean,
|
||||
) {
|
||||
|
||||
fun formatRecord(record: DestinationRecordRaw): String {
|
||||
val enrichedRecord = record.asEnrichedDestinationRecordAirbyteValue()
|
||||
|
||||
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) {
|
||||
Meta.COLUMN_NAME_AB_EXTRACTED_AT -> {
|
||||
val extractedAtMillis = (value.abValue as IntegerValue).value.longValueExact()
|
||||
outputRecord[key] = getExtractedAt(extractedAtMillis)
|
||||
}
|
||||
Meta.COLUMN_NAME_AB_META -> {
|
||||
// TODO this is a hack for T+D, we should remove it for direct-load tables
|
||||
// we're using sourceMeta instead of airbyteMeta, because the latter
|
||||
// includes changes in-connector type coercion
|
||||
// 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
|
||||
// do nothing for now - we'll be updating the meta field when we process
|
||||
// other fields in this record.
|
||||
// so we need to defer it until _after_ we process the entire record.
|
||||
}
|
||||
Meta.COLUMN_NAME_AB_RAW_ID ->
|
||||
outputRecord[key] = (value.abValue as StringValue).value
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID ->
|
||||
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()
|
||||
}
|
||||
@@ -65,6 +143,31 @@ class BigQueryRecordFormatter {
|
||||
}
|
||||
|
||||
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
|
||||
val SCHEMA_V2: Schema =
|
||||
Schema.of(
|
||||
@@ -86,5 +189,122 @@ class BigQueryRecordFormatter {
|
||||
Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64),
|
||||
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 -> {}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -16,9 +16,9 @@ data class BigqueryConfiguration(
|
||||
val datasetId: String,
|
||||
val loadingMethod: LoadingMethodConfiguration,
|
||||
val credentialsJson: String?,
|
||||
val transformationPriority: TransformationPriority,
|
||||
val rawTableDataset: String,
|
||||
val disableTypingDeduping: Boolean,
|
||||
val cdcDeletionMode: CdcDeletionMode,
|
||||
val internalTableDataset: String,
|
||||
val legacyRawTablesOnly: Boolean,
|
||||
) : DestinationConfiguration() {
|
||||
override val numOpenStreamWorkers = 3
|
||||
}
|
||||
@@ -54,14 +54,15 @@ class BigqueryConfigurationFactory :
|
||||
datasetId = pojo.datasetId,
|
||||
loadingMethodConfig,
|
||||
credentialsJson = pojo.credentialsJson,
|
||||
pojo.transformationPriority ?: TransformationPriority.INTERACTIVE,
|
||||
rawTableDataset =
|
||||
if (pojo.rawTableDataset.isNullOrBlank()) {
|
||||
// default to hard delete for backwards compatibility.
|
||||
cdcDeletionMode = pojo.cdcDeletionMode ?: CdcDeletionMode.HARD_DELETE,
|
||||
internalTableDataset =
|
||||
if (pojo.internalTableDataset.isNullOrBlank()) {
|
||||
DbConstants.DEFAULT_RAW_TABLE_NAMESPACE
|
||||
} else {
|
||||
pojo.rawTableDataset!!
|
||||
pojo.internalTableDataset!!
|
||||
},
|
||||
disableTypingDeduping = pojo.disableTypingDeduping ?: false,
|
||||
legacyRawTablesOnly = pojo.legacyRawTablesOnly ?: false,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -75,31 +75,36 @@ class BigquerySpecification : ConfigurationSpecification() {
|
||||
)
|
||||
val credentialsJson: String? = null
|
||||
|
||||
@get:JsonSchemaTitle("Transformation Query Run Type")
|
||||
@get:JsonSchemaTitle("CDC deletion mode")
|
||||
@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 don’t 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")
|
||||
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 5}""")
|
||||
val transformationPriority: TransformationPriority? = null
|
||||
|
||||
@get:JsonSchemaTitle("Raw Table Dataset Name")
|
||||
@get:JsonPropertyDescription(
|
||||
"""The dataset to write raw tables into (default: airbyte_internal)""",
|
||||
// default hard delete for backwards compatibility
|
||||
@get:JsonProperty("cdc_deletion_mode", defaultValue = "Hard delete")
|
||||
@get:JsonSchemaInject(
|
||||
json = """{"group": "sync_behavior", "order": 5, "always_show": true}""",
|
||||
)
|
||||
@get:JsonProperty("raw_data_dataset")
|
||||
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 7}""")
|
||||
val rawTableDataset: String? = null
|
||||
val cdcDeletionMode: CdcDeletionMode? = null
|
||||
|
||||
@get:JsonSchemaTitle(
|
||||
"Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)",
|
||||
"""Legacy raw tables""",
|
||||
)
|
||||
@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:JsonSchemaInject(json = """{"group": "advanced", "order": 8, "default": false}""")
|
||||
val disableTypingDeduping: Boolean? = null
|
||||
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 7, "default": false}""")
|
||||
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(
|
||||
@@ -204,6 +209,11 @@ enum class TransformationPriority(@get:JsonValue val transformationPriority: Str
|
||||
BATCH("batch")
|
||||
}
|
||||
|
||||
enum class CdcDeletionMode(@get:JsonValue val cdcDeletionMode: String) {
|
||||
HARD_DELETE("Hard delete"),
|
||||
SOFT_DELETE("Soft delete"),
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigquerySpecificationExtension : DestinationSpecificationExtension {
|
||||
override val supportedSyncModes =
|
||||
@@ -216,6 +226,7 @@ class BigquerySpecificationExtension : DestinationSpecificationExtension {
|
||||
override val groups =
|
||||
listOf(
|
||||
DestinationSpecificationExtension.Group("connection", "Connection"),
|
||||
DestinationSpecificationExtension.Group("sync_behavior", "Sync Behavior"),
|
||||
DestinationSpecificationExtension.Group("advanced", "Advanced"),
|
||||
)
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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}")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -8,20 +8,23 @@ import com.google.cloud.bigquery.*
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.LoadJobConfiguration
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.file.gcs.GcsBlob
|
||||
import io.airbyte.cdk.load.file.gcs.GcsClient
|
||||
import io.airbyte.cdk.load.message.StreamKey
|
||||
import io.airbyte.cdk.load.orchestration.db.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.TypingDedupingExecutionConfig
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.cdk.load.write.db.BulkLoader
|
||||
import io.airbyte.cdk.load.write.db.BulkLoaderFactory
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
|
||||
import io.airbyte.integrations.destination.bigquery.spec.GcsFilePostProcessing
|
||||
import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.toTableId
|
||||
import io.micronaut.context.annotation.Requires
|
||||
import io.micronaut.context.condition.Condition
|
||||
import io.micronaut.context.condition.ConditionContext
|
||||
@@ -31,11 +34,10 @@ class BigQueryBulkLoader(
|
||||
private val storageClient: GcsClient,
|
||||
private val bigQueryClient: BigQuery,
|
||||
private val bigQueryConfiguration: BigqueryConfiguration,
|
||||
private val rawTableName: TableName,
|
||||
private val rawTableSuffix: String,
|
||||
private val tableId: TableId,
|
||||
private val schema: Schema,
|
||||
) : BulkLoader<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 csvOptions =
|
||||
@@ -46,11 +48,12 @@ class BigQueryBulkLoader(
|
||||
.build()
|
||||
|
||||
val configuration =
|
||||
LoadJobConfiguration.builder(rawTableId, gcsUri)
|
||||
LoadJobConfiguration.builder(tableId, gcsUri)
|
||||
.setFormatOptions(csvOptions)
|
||||
.setSchema(BigQueryRecordFormatter.CSV_SCHEMA)
|
||||
.setSchema(schema)
|
||||
.setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
|
||||
.setJobTimeoutMs(600000L) // 10 min timeout
|
||||
.setNullMarker(BigQueryConsts.NULL_MARKER)
|
||||
.build()
|
||||
|
||||
val loadJob = bigQueryClient.create(JobInfo.of(configuration))
|
||||
@@ -59,7 +62,7 @@ class BigQueryBulkLoader(
|
||||
BigQueryUtils.waitForJobFinish(loadJob)
|
||||
} catch (e: Exception) {
|
||||
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
|
||||
)
|
||||
}
|
||||
@@ -86,11 +89,13 @@ class BigqueryConfiguredForBulkLoad : Condition {
|
||||
@Singleton
|
||||
@Requires(condition = BigqueryConfiguredForBulkLoad::class)
|
||||
class BigQueryBulkLoaderFactory(
|
||||
private val catalog: DestinationCatalog,
|
||||
private val names: TableCatalogByDescriptor,
|
||||
private val storageClient: GcsClient,
|
||||
private val bigQueryClient: BigQuery,
|
||||
private val bigQueryConfiguration: BigqueryConfiguration,
|
||||
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
private val typingDedupingStreamStateStore: StreamStateStore<TypingDedupingExecutionConfig>?,
|
||||
private val directLoadStreamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>?,
|
||||
) : BulkLoaderFactory<StreamKey, GcsBlob> {
|
||||
override val numPartWorkers: Int = 2
|
||||
override val numUploadWorkers: Int = 10
|
||||
@@ -101,12 +106,28 @@ class BigQueryBulkLoaderFactory(
|
||||
override val maxMemoryRatioReservedForParts: Double = 0.6
|
||||
|
||||
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(
|
||||
storageClient,
|
||||
bigQueryClient,
|
||||
bigQueryConfiguration,
|
||||
names[key.stream]!!.tableNames.rawTableName!!,
|
||||
streamStateStore.get(key.stream)!!.rawTableSuffix,
|
||||
tableId,
|
||||
schema,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -42,7 +42,7 @@ data class BigqueryBulkLoadConfiguration(
|
||||
ObjectStorageCompressionConfigurationProvider<BufferedOutputStream> {
|
||||
override val objectStoragePathConfiguration: ObjectStoragePathConfiguration
|
||||
override val objectStorageFormatConfiguration: ObjectStorageFormatConfiguration =
|
||||
CSVFormatConfiguration()
|
||||
CSVFormatConfiguration(rootLevelFlattening = !bigQueryConfiguration.legacyRawTablesOnly)
|
||||
override val objectStorageUploadConfiguration: ObjectStorageUploadConfiguration =
|
||||
ObjectStorageUploadConfiguration()
|
||||
override val s3BucketConfiguration: S3BucketConfiguration
|
||||
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -5,39 +5,53 @@
|
||||
package io.airbyte.integrations.destination.bigquery.write.bulk_loader
|
||||
|
||||
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.ObjectStorageFormattingWriter
|
||||
import io.airbyte.cdk.load.file.object_storage.ObjectStorageFormattingWriterFactory
|
||||
import io.airbyte.cdk.load.message.DestinationRecordRaw
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
|
||||
import jakarta.inject.Singleton
|
||||
import java.io.OutputStream
|
||||
|
||||
class BigQueryObjectStorageFormattingWriter(
|
||||
private val csvFormattingWriter: CSVFormattingWriter,
|
||||
) : ObjectStorageFormattingWriter by csvFormattingWriter {
|
||||
stream: DestinationStream,
|
||||
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) {
|
||||
csvFormattingWriter.accept(record)
|
||||
printer.printRecord(bigQueryRowGenerator.generate(record, finalSchema))
|
||||
}
|
||||
|
||||
override fun flush() {
|
||||
printer.flush()
|
||||
}
|
||||
|
||||
override fun close() {
|
||||
printer.close()
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigQueryObjectStorageFormattingWriterFactory(
|
||||
private val formatConfigProvider: ObjectStorageFormatConfigurationProvider,
|
||||
) : ObjectStorageFormattingWriterFactory {
|
||||
class BigQueryObjectStorageFormattingWriterFactory(private val config: BigqueryConfiguration) :
|
||||
ObjectStorageFormattingWriterFactory {
|
||||
override fun create(
|
||||
stream: DestinationStream,
|
||||
outputStream: OutputStream
|
||||
outputStream: OutputStream,
|
||||
): ObjectStorageFormattingWriter {
|
||||
val flatten = formatConfigProvider.objectStorageFormatConfiguration.rootLevelFlattening
|
||||
return BigQueryObjectStorageFormattingWriter(
|
||||
return if (config.legacyRawTablesOnly) {
|
||||
CSVFormattingWriter(
|
||||
stream,
|
||||
outputStream,
|
||||
rootLevelFlattening = flatten,
|
||||
rootLevelFlattening = false,
|
||||
extractedAtAsTimestampWithTimezone = true,
|
||||
),
|
||||
)
|
||||
} else {
|
||||
BigQueryObjectStorageFormattingWriter(stream, outputStream)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -9,13 +9,16 @@ import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.FormatOptions
|
||||
import com.google.cloud.bigquery.JobId
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.Schema
|
||||
import com.google.cloud.bigquery.TableDataWriteChannel
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import com.google.cloud.bigquery.WriteChannelConfiguration
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.command.DestinationCatalog
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
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.TypingDedupingExecutionConfig
|
||||
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.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.typing_deduping.toTableId
|
||||
import io.micronaut.context.annotation.Requires
|
||||
import io.micronaut.context.condition.Condition
|
||||
import io.micronaut.context.condition.ConditionContext
|
||||
@@ -40,8 +44,8 @@ class BigqueryBatchStandardInsertsLoader(
|
||||
private val bigquery: BigQuery,
|
||||
private val writeChannelConfiguration: WriteChannelConfiguration,
|
||||
private val job: JobId,
|
||||
private val recordFormatter: BigQueryRecordFormatter,
|
||||
) : DirectLoader {
|
||||
private val recordFormatter = BigQueryRecordFormatter()
|
||||
// a TableDataWriteChannel holds (by default) a 15MB buffer in memory.
|
||||
// so we start out by writing to a BAOS, which grows dynamically.
|
||||
// when the BAOS reaches 15MB, we create the TableDataWriteChannel and switch over
|
||||
@@ -115,24 +119,38 @@ class BigqueryConfiguredForBatchStandardInserts : Condition {
|
||||
@Requires(condition = BigqueryConfiguredForBatchStandardInserts::class)
|
||||
@Singleton
|
||||
class BigqueryBatchStandardInsertsLoaderFactory(
|
||||
private val catalog: DestinationCatalog,
|
||||
private val bigquery: BigQuery,
|
||||
private val config: BigqueryConfiguration,
|
||||
private val tableCatalog: TableCatalogByDescriptor,
|
||||
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
private val typingDedupingStreamStateStore: StreamStateStore<TypingDedupingExecutionConfig>?,
|
||||
private val directLoadStreamStateStore: StreamStateStore<DirectLoadTableExecutionConfig>?,
|
||||
) : DirectLoaderFactory<BigqueryBatchStandardInsertsLoader> {
|
||||
override fun create(
|
||||
streamDescriptor: DestinationStream.Descriptor,
|
||||
part: Int,
|
||||
): BigqueryBatchStandardInsertsLoader {
|
||||
val rawTableName = tableCatalog[streamDescriptor]!!.tableNames.rawTableName!!
|
||||
val rawTableNameSuffix = streamStateStore.get(streamDescriptor)!!.rawTableSuffix
|
||||
|
||||
val writeChannelConfiguration =
|
||||
WriteChannelConfiguration.newBuilder(
|
||||
TableId.of(rawTableName.namespace, rawTableName.name + rawTableNameSuffix)
|
||||
val tableId: TableId
|
||||
val schema: Schema
|
||||
val tableNameInfo = tableCatalog[streamDescriptor]!!
|
||||
if (config.legacyRawTablesOnly) {
|
||||
val rawTableName = tableNameInfo.tableNames.rawTableName!!
|
||||
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)
|
||||
.setSchema(BigQueryRecordFormatter.SCHEMA_V2)
|
||||
.setSchema(schema)
|
||||
// new-line delimited json.
|
||||
.setFormatOptions(FormatOptions.json())
|
||||
.build()
|
||||
@@ -148,6 +166,10 @@ class BigqueryBatchStandardInsertsLoaderFactory(
|
||||
bigquery,
|
||||
writeChannelConfiguration,
|
||||
jobId,
|
||||
BigQueryRecordFormatter(
|
||||
tableNameInfo.columnNameMapping,
|
||||
legacyRawTablesOnly = config.legacyRawTablesOnly,
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
/*
|
||||
* 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.BigQueryException
|
||||
@@ -12,6 +12,7 @@ import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.JobStatistics
|
||||
import com.google.cloud.bigquery.JobStatus
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.orchestration.db.DatabaseHandler
|
||||
import io.airbyte.cdk.load.orchestration.db.Sql
|
||||
@@ -21,12 +22,59 @@ import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.util.UUID
|
||||
import kotlin.math.min
|
||||
import kotlinx.coroutines.coroutineScope
|
||||
import kotlinx.coroutines.delay
|
||||
import kotlinx.coroutines.launch
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
@SuppressFBWarnings("NP_NONNULL_PARAM_VIOLATION", justification = "Kotlin is hard")
|
||||
class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocation: String) :
|
||||
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)
|
||||
override fun execute(sql: Sql) {
|
||||
val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION")
|
||||
@@ -2,7 +2,7 @@
|
||||
* 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 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.TableName
|
||||
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.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer
|
||||
import java.util.Locale
|
||||
import javax.inject.Singleton
|
||||
|
||||
private val nameTransformer = BigQuerySQLNameTransformer()
|
||||
|
||||
@Singleton
|
||||
class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTableNameGenerator {
|
||||
override fun getTableName(streamDescriptor: DestinationStream.Descriptor) =
|
||||
TableName(
|
||||
nameTransformer.getNamespace(config.rawTableDataset),
|
||||
nameTransformer.getNamespace(config.internalTableDataset),
|
||||
nameTransformer.convertStreamName(
|
||||
TypingDedupingUtil.concatenateRawTableName(
|
||||
streamDescriptor.namespace ?: config.datasetId,
|
||||
@@ -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) }
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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()
|
||||
}
|
||||
}
|
||||
@@ -2,7 +2,7 @@
|
||||
* 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.QueryJobConfiguration
|
||||
@@ -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,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -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.BigqueryConfigurationFactory
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
|
||||
import io.airbyte.integrations.destination.bigquery.spec.CdcDeletionMode
|
||||
import java.io.IOException
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Path
|
||||
@@ -34,13 +35,13 @@ object BigQueryDestinationTestUtils {
|
||||
* generated by the test.
|
||||
*
|
||||
* @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)
|
||||
fun createConfig(
|
||||
configFile: String,
|
||||
rawDatasetId: String? = null,
|
||||
disableTypingDeduping: Boolean? = null,
|
||||
cdcDeletionMode: CdcDeletionMode? = null,
|
||||
): String {
|
||||
val tmpConfigAsString = Files.readString(Path.of(configFile))
|
||||
val config = Jsons.readTree(tmpConfigAsString) as ObjectNode
|
||||
@@ -49,6 +50,9 @@ object BigQueryDestinationTestUtils {
|
||||
disableTypingDeduping?.let {
|
||||
config.put(BigQueryConsts.DISABLE_TYPE_DEDUPE, disableTypingDeduping)
|
||||
}
|
||||
cdcDeletionMode?.let {
|
||||
config.put(BigQueryConsts.CDC_DELETION_MODE, cdcDeletionMode.cdcDeletionMode)
|
||||
}
|
||||
return config.serializeToString()
|
||||
}
|
||||
|
||||
|
||||
@@ -30,8 +30,8 @@ import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.cdk.load.util.deserializeToNode
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfigurationFactory
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryRawTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryFinalTableNameGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryRawTableNameGenerator
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Reason
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
@@ -51,7 +51,8 @@ object BigqueryRawTableDataDumper : DestinationDataDumper {
|
||||
val (_, rawTableName) =
|
||||
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!!
|
||||
table.list(bigquerySchema).iterateAll().map { row ->
|
||||
OutputRecord(
|
||||
@@ -72,7 +73,7 @@ object BigqueryRawTableDataDumper : DestinationDataDumper {
|
||||
}
|
||||
?: run {
|
||||
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()
|
||||
}
|
||||
|
||||
@@ -15,16 +15,21 @@ import kotlinx.coroutines.runBlocking
|
||||
|
||||
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 {
|
||||
private val actualCleaner =
|
||||
BigqueryDestinationCleanerInstance(
|
||||
BigqueryInternalTableDatasetCleaner(
|
||||
BigQueryDestinationTestUtils.standardInsertRawOverrideConfig
|
||||
)
|
||||
.compose(
|
||||
BigqueryDestinationCleanerInstance(
|
||||
BigqueryInternalTableDatasetCleaner(
|
||||
BigQueryDestinationTestUtils.standardInsertConfig
|
||||
)
|
||||
)
|
||||
.compose(BigqueryDatasetCleaner(BigQueryDestinationTestUtils.standardInsertConfig))
|
||||
|
||||
override fun cleanup() {
|
||||
// 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() {
|
||||
val config = BigQueryDestinationTestUtils.parseConfig(configString)
|
||||
val bigquery = BigqueryBeansFactory().getBigqueryClient(config)
|
||||
|
||||
runBlocking(Dispatchers.IO) {
|
||||
logger.info { "Cleaning up old raw tables in ${config.rawTableDataset}" }
|
||||
|
||||
var rawTables = bigquery.listTables(config.rawTableDataset)
|
||||
logger.info { "Cleaning up old raw tables in ${config.internalTableDataset}" }
|
||||
var rawTables = bigquery.listTables(config.internalTableDataset)
|
||||
// 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
|
||||
// fetching new pages.
|
||||
@@ -81,7 +84,15 @@ class BigqueryDestinationCleanerInstance(private val configString: String) : Des
|
||||
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}" }
|
||||
var datasets = bigquery.listDatasets(config.projectId)
|
||||
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
|
||||
}
|
||||
}
|
||||
|
||||
@@ -4,20 +4,11 @@
|
||||
|
||||
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.ExpectedRecordMapper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
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.RootLevelTimestampsToUtcMapper
|
||||
import io.airbyte.cdk.load.toolkits.load.db.orchestration.TypingDedupingMetaChangeMapper
|
||||
import io.airbyte.cdk.load.write.AllTypesBehavior
|
||||
import io.airbyte.cdk.load.write.BasicFunctionalityIntegrationTest
|
||||
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.STANDARD_INSERT_CONFIG
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryColumnNameGenerator
|
||||
import kotlin.test.assertEquals
|
||||
import org.junit.jupiter.api.Assertions.assertTrue
|
||||
import io.airbyte.integrations.destination.bigquery.spec.CdcDeletionMode
|
||||
import io.airbyte.integrations.destination.bigquery.write.typing_deduping.BigqueryColumnNameGenerator
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.assertAll
|
||||
|
||||
abstract class BigqueryWriteTest(
|
||||
configContents: String,
|
||||
@@ -42,6 +31,7 @@ abstract class BigqueryWriteTest(
|
||||
expectedRecordMapper: ExpectedRecordMapper,
|
||||
isStreamSchemaRetroactive: Boolean,
|
||||
preserveUndeclaredFields: Boolean,
|
||||
commitDataIncrementallyToEmptyDestination: Boolean,
|
||||
dedupBehavior: DedupBehavior?,
|
||||
nullEqualsUnset: Boolean,
|
||||
allTypesBehavior: AllTypesBehavior,
|
||||
@@ -61,6 +51,7 @@ abstract class BigqueryWriteTest(
|
||||
preserveUndeclaredFields = preserveUndeclaredFields,
|
||||
supportFileTransfer = false,
|
||||
commitDataIncrementally = false,
|
||||
commitDataIncrementallyToEmptyDestination = commitDataIncrementallyToEmptyDestination,
|
||||
allTypesBehavior = allTypesBehavior,
|
||||
nullEqualsUnset = nullEqualsUnset,
|
||||
configUpdater = BigqueryConfigUpdater,
|
||||
@@ -76,22 +67,41 @@ abstract class BigqueryRawTablesWriteTest(
|
||||
UncoercedExpectedRecordMapper,
|
||||
isStreamSchemaRetroactive = false,
|
||||
preserveUndeclaredFields = true,
|
||||
commitDataIncrementallyToEmptyDestination = false,
|
||||
dedupBehavior = null,
|
||||
nullEqualsUnset = false,
|
||||
Untyped,
|
||||
)
|
||||
|
||||
abstract class BigqueryTDWriteTest(configContents: String) :
|
||||
abstract class BigqueryDirectLoadWriteTest(
|
||||
configContents: String,
|
||||
cdcDeletionMode: CdcDeletionMode,
|
||||
) :
|
||||
BigqueryWriteTest(
|
||||
configContents = configContents,
|
||||
BigqueryFinalTableDataDumper,
|
||||
ColumnNameModifyingMapper(BigqueryColumnNameGenerator())
|
||||
.compose(TimeWithTimezoneMapper)
|
||||
.compose(RootLevelTimestampsToUtcMapper)
|
||||
.compose(TypingDedupingMetaChangeMapper)
|
||||
.compose(IntegralNumberRecordMapper),
|
||||
isStreamSchemaRetroactive = true,
|
||||
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,
|
||||
StronglyTyped(
|
||||
convertAllValuesToString = true,
|
||||
@@ -99,247 +109,66 @@ abstract class BigqueryTDWriteTest(configContents: String) :
|
||||
nestedFloatLosesPrecision = true,
|
||||
integerCanBeLarge = false,
|
||||
numberCanBeLarge = false,
|
||||
timeWithTimezoneBehavior = SimpleValueBehavior.PASS_THROUGH,
|
||||
numberIsFixedPointPrecision38Scale9 = true,
|
||||
timeWithTimezoneBehavior = SimpleValueBehavior.STRONGLY_TYPE,
|
||||
),
|
||||
) {
|
||||
private val oldCdkDestinationFactory =
|
||||
DockerizedDestinationFactory("airbyte/destination-bigquery", "2.10.2")
|
||||
|
||||
@Test
|
||||
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 :
|
||||
class StandardInsertRawOverrideRawTables :
|
||||
BigqueryRawTablesWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
configFile = STANDARD_INSERT_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
disableTypingDeduping = true,
|
||||
),
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
@Test
|
||||
override fun testAppendSchemaEvolution() {
|
||||
super.testAppendSchemaEvolution()
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
class StandardInsertRawOverride :
|
||||
BigqueryTDWriteTest(BigQueryDestinationTestUtils.standardInsertRawOverrideConfig) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
@Test
|
||||
override fun testFunkyCharacters() {
|
||||
super.testFunkyCharacters()
|
||||
}
|
||||
}
|
||||
BigqueryDirectLoadWriteTest(
|
||||
BigQueryDestinationTestUtils.standardInsertRawOverrideConfig,
|
||||
CdcDeletionMode.HARD_DELETE,
|
||||
)
|
||||
|
||||
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
|
||||
override fun testDedup() {
|
||||
super.testDedup()
|
||||
}
|
||||
}
|
||||
|
||||
class GcsRawOverrideDisableTd :
|
||||
class GcsRawOverrideRawTables :
|
||||
BigqueryRawTablesWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
configFile = GCS_STAGING_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
disableTypingDeduping = true,
|
||||
),
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
)
|
||||
|
||||
class GcsRawOverride :
|
||||
BigqueryTDWriteTest(
|
||||
BigqueryDirectLoadWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
configFile = GCS_STAGING_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
),
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
CdcDeletionMode.HARD_DELETE,
|
||||
)
|
||||
|
||||
class Gcs :
|
||||
BigqueryTDWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(configFile = GCS_STAGING_CONFIG)
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
BigqueryDirectLoadWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(configFile = GCS_STAGING_CONFIG),
|
||||
CdcDeletionMode.HARD_DELETE,
|
||||
)
|
||||
|
||||
@@ -10,8 +10,11 @@ import io.airbyte.cdk.load.data.ArrayValue
|
||||
import io.airbyte.cdk.load.data.IntegerValue
|
||||
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.TimeWithTimezoneValue
|
||||
import io.airbyte.cdk.load.test.util.ExpectedRecordMapper
|
||||
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
|
||||
@@ -56,3 +59,23 @@ object IntegralNumberRecordMapper : ExpectedRecordMapper {
|
||||
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)
|
||||
}
|
||||
}
|
||||
@@ -129,35 +129,39 @@
|
||||
"airbyte_secret" : true,
|
||||
"always_show" : true
|
||||
},
|
||||
"transformation_priority" : {
|
||||
"cdc_deletion_mode" : {
|
||||
"type" : "string",
|
||||
"default" : "interactive",
|
||||
"enum" : [ "interactive", "batch" ],
|
||||
"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 don’t 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.",
|
||||
"title" : "Transformation Query Run Type",
|
||||
"group" : "advanced",
|
||||
"order" : 5
|
||||
},
|
||||
"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
|
||||
"default" : "Hard delete",
|
||||
"enum" : [ "Hard delete", "Soft delete" ],
|
||||
"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" : "CDC deletion mode",
|
||||
"group" : "sync_behavior",
|
||||
"order" : 5,
|
||||
"always_show" : true
|
||||
},
|
||||
"disable_type_dedupe" : {
|
||||
"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",
|
||||
"title" : "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)",
|
||||
"description" : "Write the legacy \"raw tables\" format, to enable backwards compatibility with older versions of this connector.",
|
||||
"title" : "Legacy raw tables",
|
||||
"group" : "advanced",
|
||||
"order" : 8,
|
||||
"order" : 7,
|
||||
"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" ],
|
||||
"groups" : [ {
|
||||
"id" : "connection",
|
||||
"title" : "Connection"
|
||||
}, {
|
||||
"id" : "sync_behavior",
|
||||
"title" : "Sync Behavior"
|
||||
}, {
|
||||
"id" : "advanced",
|
||||
"title" : "Advanced"
|
||||
|
||||
@@ -129,35 +129,39 @@
|
||||
"airbyte_secret" : true,
|
||||
"always_show" : true
|
||||
},
|
||||
"transformation_priority" : {
|
||||
"cdc_deletion_mode" : {
|
||||
"type" : "string",
|
||||
"default" : "interactive",
|
||||
"enum" : [ "interactive", "batch" ],
|
||||
"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 don’t 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.",
|
||||
"title" : "Transformation Query Run Type",
|
||||
"group" : "advanced",
|
||||
"order" : 5
|
||||
},
|
||||
"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
|
||||
"default" : "Hard delete",
|
||||
"enum" : [ "Hard delete", "Soft delete" ],
|
||||
"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" : "CDC deletion mode",
|
||||
"group" : "sync_behavior",
|
||||
"order" : 5,
|
||||
"always_show" : true
|
||||
},
|
||||
"disable_type_dedupe" : {
|
||||
"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",
|
||||
"title" : "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)",
|
||||
"description" : "Write the legacy \"raw tables\" format, to enable backwards compatibility with older versions of this connector.",
|
||||
"title" : "Legacy raw tables",
|
||||
"group" : "advanced",
|
||||
"order" : 8,
|
||||
"order" : 7,
|
||||
"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" ],
|
||||
"groups" : [ {
|
||||
"id" : "connection",
|
||||
"title" : "Connection"
|
||||
}, {
|
||||
"id" : "sync_behavior",
|
||||
"title" : "Sync Behavior"
|
||||
}, {
|
||||
"id" : "advanced",
|
||||
"title" : "Advanced"
|
||||
|
||||
@@ -1,7 +1,7 @@
|
||||
/*
|
||||
* 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.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.UnionType
|
||||
import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.toDialectType
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDatabaseInitialStatusGatherer.Companion.clusteringMatches
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigqueryDatabaseInitialStatusGatherer.Companion.partitioningMatches
|
||||
import io.airbyte.cdk.load.orchestration.db.DefaultTempTableNameGenerator
|
||||
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.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.Test
|
||||
import org.mockito.Mockito
|
||||
import org.mockito.Mockito.RETURNS_DEEP_STUBS
|
||||
|
||||
class BigqueryDestinationHandlerTest {
|
||||
class BigqueryDirectLoadNativeTableOperationsTest {
|
||||
@Test
|
||||
fun testToDialectType() {
|
||||
val s = ObjectType(linkedMapOf())
|
||||
@@ -70,12 +74,30 @@ class BigqueryDestinationHandlerTest {
|
||||
Mockito.`when`(existingTable.schema!!.fields)
|
||||
.thenReturn(FieldList.of(Field.of("a2", StandardSQLTypeName.INT64)))
|
||||
val alterTableReport =
|
||||
BigqueryDatabaseInitialStatusGatherer(Mockito.mock())
|
||||
BigqueryDirectLoadNativeTableOperations(
|
||||
Mockito.mock(),
|
||||
Mockito.mock(),
|
||||
Mockito.mock(),
|
||||
projectId = "unused",
|
||||
tempTableNameGenerator = DefaultTempTableNameGenerator("unused"),
|
||||
)
|
||||
.buildAlterTableReport(stream, columnNameMapping, existingTable)
|
||||
Assertions.assertAll(
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToAdd) },
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToRemove) },
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToChangeType) },
|
||||
{
|
||||
Assertions.assertEquals(
|
||||
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 =
|
||||
BigqueryDatabaseInitialStatusGatherer(Mockito.mock())
|
||||
BigqueryDirectLoadNativeTableOperations(
|
||||
Mockito.mock(),
|
||||
Mockito.mock(),
|
||||
Mockito.mock(),
|
||||
projectId = "unused",
|
||||
tempTableNameGenerator = DefaultTempTableNameGenerator("unused"),
|
||||
)
|
||||
.buildAlterTableReport(stream, columnNameMapping, existingTable)
|
||||
// NB: column names in AlterTableReport are all _after_ destination name transform
|
||||
Assertions.assertAll(
|
||||
{ Assertions.assertEquals(setOf("c2"), alterTableReport.columnsToAdd) },
|
||||
{ Assertions.assertEquals(setOf("b2"), alterTableReport.columnsToRemove) },
|
||||
{ Assertions.assertEquals(setOf("a2"), alterTableReport.columnsToChangeType) },
|
||||
{
|
||||
Assertions.assertEquals(
|
||||
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
|
||||
)
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
@@ -191,6 +191,7 @@ class AirbyteBulkConnectorPlugin implements Plugin<Project> {
|
||||
testLogging() {
|
||||
events 'skipped', 'started', 'passed', 'failed'
|
||||
exceptionFormat 'full'
|
||||
showStandardStreams = true
|
||||
}
|
||||
|
||||
// Always re-run integration tests no matter what.
|
||||
|
||||
@@ -1,5 +1,17 @@
|
||||
# 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
|
||||
|
||||
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.
|
||||
|
||||
@@ -115,28 +115,6 @@ The BigQuery destination connector supports the following
|
||||
|
||||
## 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:
|
||||
- `airbyte_raw_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
|
||||
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
|
||||
|
||||
Follow
|
||||
@@ -213,6 +210,7 @@ tutorials:
|
||||
|
||||
| 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-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 |
|
||||
|
||||
Reference in New Issue
Block a user