[BigQuery] 2.12.3 release. (#61648)
Co-authored-by: Edward Gao <edward.gao@airbyte.io>
This commit is contained in:
@@ -1,14 +1,20 @@
|
||||
plugins {
|
||||
id 'application'
|
||||
id 'airbyte-bulk-connector'
|
||||
id 'airbyte-java-connector'
|
||||
id "io.airbyte.gradle.docker"
|
||||
id 'airbyte-connector-docker-convention'
|
||||
}
|
||||
|
||||
airbyteBulkConnector {
|
||||
core = 'load'
|
||||
toolkits = ['load-gcs', 'load-db', 'load-s3']
|
||||
cdk = 'local'
|
||||
airbyteJavaConnector {
|
||||
cdkVersionRequired = '0.48.8'
|
||||
features = [
|
||||
'db-destinations',
|
||||
'datastore-bigquery',
|
||||
'typing-deduping',
|
||||
'gcs-destinations',
|
||||
'core',
|
||||
]
|
||||
useLocalCdk = false
|
||||
}
|
||||
|
||||
java {
|
||||
@@ -34,7 +40,5 @@ application {
|
||||
}
|
||||
|
||||
dependencies {
|
||||
implementation 'com.google.cloud:google-cloud-bigquery'
|
||||
|
||||
integrationTestImplementation 'com.google.cloud:google-cloud-bigquery'
|
||||
implementation 'org.apache.commons:commons-text:1.12.0'
|
||||
}
|
||||
|
||||
@@ -2,11 +2,10 @@ data:
|
||||
ab_internal:
|
||||
ql: 300
|
||||
sl: 300
|
||||
requireVersionIncrementsInPullRequests: false
|
||||
connectorSubtype: database
|
||||
connectorType: destination
|
||||
definitionId: 22f6c74f-5699-40ff-833c-4a879ea40133
|
||||
dockerImageTag: 2.12.2-rc.1
|
||||
dockerImageTag: 2.12.3
|
||||
dockerRepository: airbyte/destination-bigquery
|
||||
documentationUrl: https://docs.airbyte.com/integrations/destinations/bigquery
|
||||
githubIssueLabel: destination-bigquery
|
||||
@@ -17,11 +16,8 @@ data:
|
||||
baseImage: docker.io/airbyte/java-connector-base:2.0.1@sha256:ec89bd1a89e825514dd2fc8730ba299a3ae1544580a078df0e35c5202c2085b3
|
||||
registryOverrides:
|
||||
cloud:
|
||||
# rolling back b/c of spurious "no stream status" error
|
||||
dockerImageTag: 2.10.2
|
||||
enabled: true
|
||||
oss:
|
||||
dockerImageTag: 2.10.2
|
||||
enabled: true
|
||||
releaseStage: generally_available
|
||||
releases:
|
||||
@@ -29,8 +25,6 @@ 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"
|
||||
rolloutConfiguration:
|
||||
enableProgressiveRollout: true
|
||||
resourceRequirements:
|
||||
jobSpecific:
|
||||
- jobType: sync
|
||||
@@ -46,31 +40,66 @@ data:
|
||||
- suite: unitTests
|
||||
- suite: integrationTests
|
||||
testSecrets:
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_DISABLETD_GCS_RAW_OVERRIDE
|
||||
fileName: credentials-1s1t-disabletd-gcs-raw-override.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_DISABLETD_STANDARD_OVERRIDE
|
||||
fileName: credentials-1s1t-disabletd-standard-raw-override.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_GCS
|
||||
fileName: credentials-1s1t-gcs.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_GCS_RAW_OVERRIDE
|
||||
fileName: credentials-1s1t-gcs-raw-override.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_STANDARD
|
||||
fileName: credentials-1s1t-standard.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_STANDARD_RAW_OVERRIDE
|
||||
fileName: credentials-1s1t-standard-raw-override.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_BAD_PROJECT_CREDS
|
||||
fileName: credentials-badproject.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_GCS_STAGING
|
||||
fileName: credentials-gcs-staging.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_NO_PUBLIC_SCHEMA_EDIT_ROLE
|
||||
fileName: credentials-no-edit-public-schema-role.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_STANDARD
|
||||
fileName: credentials-standard.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_STANDARD-NO-DATASET-CREATION__CREDS
|
||||
fileName: credentials-standard-no-dataset-creation.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_STANDARD_NON_BILLABLE_PROJECT__CREDS
|
||||
fileName: credentials-standard-non-billable-project.json
|
||||
secretStore:
|
||||
type: GSM
|
||||
alias: airbyte-connector-testing-secret-store
|
||||
- name: SECRET_DESTINATION-BIGQUERY_CREDENTIALS_1S1T_GCS_BAD_COPY
|
||||
fileName: credentials-1s1t-gcs-bad-copy-permission.json
|
||||
secretStore:
|
||||
|
||||
@@ -1,3 +1,3 @@
|
||||
include = [
|
||||
"${POE_GIT_DIR}/poe-tasks/gradle-connector-tasks.toml",
|
||||
]
|
||||
]
|
||||
@@ -0,0 +1,67 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.integrations.destination.async.AsyncStreamConsumer
|
||||
import io.airbyte.cdk.integrations.destination.async.buffers.BufferManager
|
||||
import io.airbyte.cdk.integrations.destination.operation.SyncOperation
|
||||
import io.airbyte.integrations.base.destination.operation.DefaultFlush
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
|
||||
import java.util.*
|
||||
import java.util.function.Consumer
|
||||
|
||||
object BigQueryConsumerFactory {
|
||||
|
||||
fun createStagingConsumer(
|
||||
outputRecordCollector: Consumer<AirbyteMessage>,
|
||||
syncOperation: SyncOperation,
|
||||
catalog: ConfiguredAirbyteCatalog,
|
||||
defaultNamespace: String
|
||||
): AsyncStreamConsumer {
|
||||
// values here are resurrected from some old code.
|
||||
// TODO: Find why max memory ratio is 0.4 capped
|
||||
return AsyncStreamConsumer(
|
||||
outputRecordCollector = outputRecordCollector,
|
||||
onStart = {},
|
||||
onClose = { _, streamSyncSummaries ->
|
||||
syncOperation.finalizeStreams(streamSyncSummaries)
|
||||
},
|
||||
onFlush = DefaultFlush(200 * 1024 * 1024, syncOperation),
|
||||
catalog = catalog,
|
||||
bufferManager =
|
||||
BufferManager(
|
||||
defaultNamespace,
|
||||
(Runtime.getRuntime().maxMemory() * 0.4).toLong(),
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
fun createDirectUploadConsumer(
|
||||
outputRecordCollector: Consumer<AirbyteMessage>,
|
||||
syncOperation: SyncOperation,
|
||||
catalog: ConfiguredAirbyteCatalog,
|
||||
defaultNamespace: String
|
||||
): AsyncStreamConsumer {
|
||||
|
||||
// TODO: Why is Standard consumer operating at memory ratio of 0.5
|
||||
// and Max 2 threads and some weird 20% max memory as the default flush size.
|
||||
return AsyncStreamConsumer(
|
||||
outputRecordCollector = outputRecordCollector,
|
||||
onStart = {},
|
||||
onClose = { _, streamSyncSummaries ->
|
||||
syncOperation.finalizeStreams(streamSyncSummaries)
|
||||
},
|
||||
onFlush =
|
||||
DefaultFlush((Runtime.getRuntime().maxMemory() * 0.2).toLong(), syncOperation),
|
||||
catalog = catalog,
|
||||
bufferManager =
|
||||
BufferManager(
|
||||
defaultNamespace,
|
||||
(Runtime.getRuntime().maxMemory() * 0.5).toLong(),
|
||||
),
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -3,17 +3,566 @@
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.AirbyteDestinationRunner
|
||||
import io.airbyte.cdk.load.command.aws.AwsToolkitConstants
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.google.auth.oauth2.GoogleCredentials
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.BigQueryOptions
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import com.google.cloud.storage.Storage
|
||||
import com.google.cloud.storage.StorageOptions
|
||||
import com.google.common.base.Charsets
|
||||
import io.airbyte.cdk.integrations.BaseConnector
|
||||
import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addAllStringsInConfigForDeinterpolation
|
||||
import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler.Companion.addThrowableForDeinterpolation
|
||||
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
|
||||
import io.airbyte.cdk.integrations.base.Destination
|
||||
import io.airbyte.cdk.integrations.base.IntegrationRunner
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants.DestinationColumns.*
|
||||
import io.airbyte.cdk.integrations.base.SerializedAirbyteMessageConsumer
|
||||
import io.airbyte.cdk.integrations.base.TypingAndDedupingFlag.getRawNamespaceOverride
|
||||
import io.airbyte.cdk.integrations.destination.StreamSyncSummary
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage
|
||||
import io.airbyte.cdk.integrations.destination.gcs.BaseGcsDestination
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations
|
||||
import io.airbyte.cdk.integrations.destination.operation.SyncOperation
|
||||
import io.airbyte.cdk.integrations.destination.s3.FileUploadFormat
|
||||
import io.airbyte.cdk.integrations.destination.staging.operation.StagingStreamOperations
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException
|
||||
import io.airbyte.commons.json.Jsons.serialize
|
||||
import io.airbyte.commons.json.Jsons.tryDeserialize
|
||||
import io.airbyte.integrations.base.destination.operation.DefaultSyncOperation
|
||||
import io.airbyte.integrations.base.destination.operation.StandardStreamOperation
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.CatalogParser
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.ImportType
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.InitialRawTableStatus
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.ParsedCatalog
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.Sql
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsts as bqConstants
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsumerFactory.createDirectUploadConsumer
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsumerFactory.createStagingConsumer
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDV2Migration
|
||||
import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState
|
||||
import io.airbyte.integrations.destination.bigquery.migrators.BigqueryAirbyteMetaAndGenerationIdMigration
|
||||
import io.airbyte.integrations.destination.bigquery.operation.BigQueryDirectLoadingStorageOperation
|
||||
import io.airbyte.integrations.destination.bigquery.operation.BigQueryGcsStorageOperation
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.airbyte.protocol.models.v0.AirbyteConnectionStatus
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta
|
||||
import io.airbyte.protocol.models.v0.AirbyteStreamStatusTraceMessage
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteCatalog
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.io.ByteArrayInputStream
|
||||
import java.io.IOException
|
||||
import java.util.*
|
||||
import java.util.function.Consumer
|
||||
|
||||
/**
|
||||
* This is needed because the GCS client is, under the hood, using the S3Client.
|
||||
*
|
||||
* And the S3Client depends on the AWS environment - we're not actually _using the assume role
|
||||
* stuff, but the wiring needs to be satisfied.
|
||||
*/
|
||||
val additionalMicronautEnvs = listOf(AwsToolkitConstants.MICRONAUT_ENVIRONMENT)
|
||||
private val log = KotlinLogging.logger {}
|
||||
|
||||
class BigQueryDestination : BaseConnector(), Destination {
|
||||
|
||||
override fun check(config: JsonNode): AirbyteConnectionStatus? {
|
||||
|
||||
try {
|
||||
|
||||
val datasetId = BigQueryUtils.getDatasetId(config)
|
||||
val datasetLocation = BigQueryUtils.getDatasetLocation(config)
|
||||
val bigquery = getBigQuery(config)
|
||||
val uploadingMethod = BigQueryUtils.getLoadingMethod(config)
|
||||
|
||||
val dataset = BigQueryUtils.getOrCreateDataset(bigquery, datasetId, datasetLocation)
|
||||
if (dataset.location != datasetLocation) {
|
||||
throw ConfigErrorException(
|
||||
"Actual dataset location doesn't match to location from config",
|
||||
)
|
||||
}
|
||||
|
||||
val queryConfig =
|
||||
QueryJobConfiguration.newBuilder(
|
||||
String.format(
|
||||
"SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES` LIMIT 1;",
|
||||
datasetId,
|
||||
),
|
||||
)
|
||||
.setUseLegacySql(false)
|
||||
.build()
|
||||
|
||||
val result = BigQueryUtils.executeQuery(bigquery, queryConfig)
|
||||
|
||||
if (result.getLeft() == null) {
|
||||
|
||||
return AirbyteConnectionStatus()
|
||||
.withStatus(AirbyteConnectionStatus.Status.FAILED)
|
||||
.withMessage(result.right)
|
||||
}
|
||||
|
||||
if (UploadingMethod.GCS == uploadingMethod) {
|
||||
|
||||
val gcsStatus = checkGcsAccessPermission(config)
|
||||
if (gcsStatus!!.status != AirbyteConnectionStatus.Status.SUCCEEDED) {
|
||||
return gcsStatus
|
||||
}
|
||||
|
||||
// Copy a temporary dataset to confirm copy permissions are working
|
||||
val bigQueryStatus = checkBigQueryCopyPermission(config)
|
||||
|
||||
if (bigQueryStatus.status != AirbyteConnectionStatus.Status.SUCCEEDED) {
|
||||
return bigQueryStatus
|
||||
}
|
||||
}
|
||||
|
||||
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
|
||||
} catch (e: Exception) {
|
||||
log.error(e) { "Check failed." }
|
||||
throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method performs a copy operation to copy data into a temporary table on BigQuery to
|
||||
* check if the existing permissions are sufficient to copy data. If the permissions are not
|
||||
* sufficient, then an exception is thrown with a message showing the missing permission
|
||||
*/
|
||||
private fun checkBigQueryCopyPermission(config: JsonNode): AirbyteConnectionStatus {
|
||||
|
||||
// TODO: Need to add a step in this method to first check permissions
|
||||
// using testIamPermissions before trying the actual copying of data
|
||||
// Created issue for tracking:
|
||||
// https://github.com/airbytehq/airbyte-internal-issues/issues/8888
|
||||
|
||||
val datasetLocation = BigQueryUtils.getDatasetLocation(config)
|
||||
val bigquery = getBigQuery(config)
|
||||
|
||||
val gcsNameTransformer = GcsNameTransformer()
|
||||
val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config)
|
||||
val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config)
|
||||
val gcsOperations =
|
||||
GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig)
|
||||
|
||||
val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText()
|
||||
val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation)
|
||||
val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation)
|
||||
val defaultDataset = BigQueryUtils.getDatasetId(config)
|
||||
|
||||
val finalTableName =
|
||||
"_airbyte_bigquery_connection_test_" +
|
||||
UUID.randomUUID().toString().replace("-".toRegex(), "")
|
||||
|
||||
val rawDatasetOverride: String =
|
||||
if (getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).isPresent) {
|
||||
getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET).get()
|
||||
} else {
|
||||
JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE
|
||||
}
|
||||
|
||||
val streamId =
|
||||
sqlGenerator.buildStreamId(defaultDataset, finalTableName, rawDatasetOverride)
|
||||
|
||||
try {
|
||||
|
||||
// Copy a dataset into a BigQuery table to confirm the copy operation is working
|
||||
// correctly with the existing permissions
|
||||
|
||||
val streamConfig =
|
||||
StreamConfig(
|
||||
id = streamId,
|
||||
postImportAction = ImportType.APPEND,
|
||||
primaryKey = listOf(),
|
||||
cursor = Optional.empty(),
|
||||
columns = linkedMapOf(),
|
||||
generationId = 1,
|
||||
minimumGenerationId = 1,
|
||||
syncId = 0
|
||||
)
|
||||
|
||||
// None of the fields in destination initial status matter
|
||||
// for a dummy sync with type-dedupe disabled. We only look at these
|
||||
// when we perform final table related setup operations.
|
||||
// We just need the streamId to perform the calls in streamOperation.
|
||||
|
||||
val initialStatus =
|
||||
DestinationInitialStatus(
|
||||
streamConfig = streamConfig,
|
||||
isFinalTablePresent = false,
|
||||
initialRawTableStatus =
|
||||
InitialRawTableStatus(
|
||||
rawTableExists = false,
|
||||
hasUnprocessedRecords = true,
|
||||
maxProcessedTimestamp = Optional.empty()
|
||||
),
|
||||
initialTempRawTableStatus =
|
||||
InitialRawTableStatus(
|
||||
rawTableExists = false,
|
||||
hasUnprocessedRecords = true,
|
||||
maxProcessedTimestamp = Optional.empty()
|
||||
),
|
||||
isSchemaMismatch = true,
|
||||
isFinalTableEmpty = true,
|
||||
destinationState = BigQueryDestinationState(needsSoftReset = false),
|
||||
finalTempTableGenerationId = null,
|
||||
finalTableGenerationId = null,
|
||||
)
|
||||
|
||||
// We simulate a mini-sync to see the raw table code path is exercised. and disable T+D
|
||||
destinationHandler.createNamespaces(setOf(defaultDataset, rawDatasetOverride))
|
||||
|
||||
val bigQueryGcsStorageOperations =
|
||||
BigQueryGcsStorageOperation(
|
||||
gcsOperations,
|
||||
gcsConfig,
|
||||
gcsNameTransformer,
|
||||
keepStagingFiles,
|
||||
bigquery,
|
||||
sqlGenerator,
|
||||
destinationHandler,
|
||||
)
|
||||
|
||||
val streamOperation: StagingStreamOperations<BigQueryDestinationState> =
|
||||
StagingStreamOperations(
|
||||
bigQueryGcsStorageOperations,
|
||||
initialStatus,
|
||||
FileUploadFormat.CSV,
|
||||
V2_WITH_GENERATION,
|
||||
disableTypeDedupe = true
|
||||
)
|
||||
|
||||
// Dummy message
|
||||
val data =
|
||||
"""
|
||||
{"testKey": "testValue"}
|
||||
""".trimIndent()
|
||||
|
||||
val message =
|
||||
PartialAirbyteMessage()
|
||||
.withSerialized(data)
|
||||
.withRecord(
|
||||
PartialAirbyteRecordMessage()
|
||||
.withEmittedAt(System.currentTimeMillis())
|
||||
.withMeta(
|
||||
AirbyteRecordMessageMeta(),
|
||||
),
|
||||
)
|
||||
|
||||
streamOperation.writeRecords(streamConfig, listOf(message).stream())
|
||||
streamOperation.finalizeTable(
|
||||
streamConfig,
|
||||
StreamSyncSummary(1, AirbyteStreamStatusTraceMessage.AirbyteStreamStatus.COMPLETE),
|
||||
)
|
||||
|
||||
// Note: Deletion of the temporary table is being done in the finally block below
|
||||
|
||||
return AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
|
||||
} catch (e: Exception) {
|
||||
|
||||
log.error(e) { "checkGcsCopyPermission failed." }
|
||||
|
||||
throw ConfigErrorException((if (e.message != null) e.message else e.toString())!!)
|
||||
} finally {
|
||||
|
||||
try {
|
||||
// In the finally block, clean up the raw table
|
||||
// If there was an exception in the flow above, then the table may still exist
|
||||
destinationHandler.execute(
|
||||
Sql.of(
|
||||
"DROP TABLE IF EXISTS $projectId.${streamId.rawNamespace}.${streamId.rawName};",
|
||||
),
|
||||
)
|
||||
} catch (e: Exception) {
|
||||
log.error(e) {
|
||||
"Error while cleaning up the temporary table... not throwing a new exception"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* This method does two checks: 1) permissions related to the bucket, and 2) the ability to
|
||||
* create and delete an actual file. The latter is important because even if the service account
|
||||
* may have the proper permissions, the HMAC keys can only be verified by running the actual GCS
|
||||
* check.
|
||||
*/
|
||||
private fun checkGcsAccessPermission(config: JsonNode): AirbyteConnectionStatus? {
|
||||
val loadingMethod = config[bqConstants.LOADING_METHOD]
|
||||
val bucketName = loadingMethod[bqConstants.GCS_BUCKET_NAME].asText()
|
||||
val missingPermissions: MutableList<String> = ArrayList()
|
||||
|
||||
try {
|
||||
val credentials = getServiceAccountCredentials(config)
|
||||
val storage: Storage =
|
||||
StorageOptions.newBuilder()
|
||||
.setProjectId(config[bqConstants.CONFIG_PROJECT_ID].asText())
|
||||
.setCredentials(credentials)
|
||||
.setHeaderProvider(BigQueryUtils.headerProvider)
|
||||
.build()
|
||||
.service
|
||||
val permissionsCheckStatusList: List<Boolean> =
|
||||
storage.testIamPermissions(bucketName, REQUIRED_GCS_PERMISSIONS)
|
||||
|
||||
// testIamPermissions returns a list of booleans
|
||||
// in the same order of the presented permissions list
|
||||
missingPermissions.addAll(
|
||||
permissionsCheckStatusList
|
||||
.asSequence()
|
||||
.withIndex()
|
||||
.filter { !it.value }
|
||||
.map { REQUIRED_GCS_PERMISSIONS[it.index] }
|
||||
.toList(),
|
||||
)
|
||||
|
||||
val gcsDestination: BaseGcsDestination = object : BaseGcsDestination() {}
|
||||
val gcsJsonNodeConfig = BigQueryUtils.getGcsJsonNodeConfig(config)
|
||||
return gcsDestination.check(gcsJsonNodeConfig)
|
||||
} catch (e: Exception) {
|
||||
val message = StringBuilder("Cannot access the GCS bucket.")
|
||||
if (!missingPermissions.isEmpty()) {
|
||||
message
|
||||
.append(" The following permissions are missing on the service account: ")
|
||||
.append(java.lang.String.join(", ", missingPermissions))
|
||||
.append(".")
|
||||
}
|
||||
message.append(
|
||||
" Please make sure the service account can access the bucket path, and the HMAC keys are correct.",
|
||||
)
|
||||
|
||||
log.error(e) { message.toString() }
|
||||
throw ConfigErrorException(
|
||||
"Could not access the GCS bucket with the provided configuration.\n",
|
||||
e,
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns a [AirbyteMessageConsumer] based on whether the uploading mode is STANDARD INSERTS or
|
||||
* using STAGING
|
||||
*
|
||||
* @param config
|
||||
* - integration-specific configuration object as json. e.g. { "username": "airbyte",
|
||||
* "password": "super secure" }
|
||||
* @param catalog
|
||||
* - schema of the incoming messages.
|
||||
*/
|
||||
override fun getConsumer(
|
||||
config: JsonNode,
|
||||
catalog: ConfiguredAirbyteCatalog,
|
||||
outputRecordCollector: Consumer<AirbyteMessage>
|
||||
): AirbyteMessageConsumer? {
|
||||
throw UnsupportedOperationException("Should use getSerializedMessageConsumer")
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun getSerializedMessageConsumer(
|
||||
config: JsonNode,
|
||||
catalog: ConfiguredAirbyteCatalog,
|
||||
outputRecordCollector: Consumer<AirbyteMessage>
|
||||
): SerializedAirbyteMessageConsumer {
|
||||
val uploadingMethod = BigQueryUtils.getLoadingMethod(config)
|
||||
val defaultNamespace = BigQueryUtils.getDatasetId(config)
|
||||
val disableTypeDedupe = BigQueryUtils.getDisableTypeDedupFlag(config)
|
||||
val datasetLocation = BigQueryUtils.getDatasetLocation(config)
|
||||
val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText()
|
||||
val bigquery = getBigQuery(config)
|
||||
val rawNamespaceOverride = getRawNamespaceOverride(bqConstants.RAW_DATA_DATASET)
|
||||
|
||||
addAllStringsInConfigForDeinterpolation(config)
|
||||
val serviceAccountKey = config[bqConstants.CONFIG_CREDS]
|
||||
if (serviceAccountKey != null) {
|
||||
// If the service account key is a non-null string, we will try to
|
||||
// deserialize it. Otherwise, we will let the Google library find it in
|
||||
// the environment during the client initialization.
|
||||
if (serviceAccountKey.isTextual) {
|
||||
// There are cases where we fail to deserialize the service account key. In these
|
||||
// cases, we
|
||||
// shouldn't do anything.
|
||||
// Google's creds library is more lenient with JSON-parsing than Jackson, and I'd
|
||||
// rather just let it
|
||||
// go.
|
||||
tryDeserialize(serviceAccountKey.asText()).ifPresent { obj: JsonNode ->
|
||||
addAllStringsInConfigForDeinterpolation(obj)
|
||||
}
|
||||
} else {
|
||||
addAllStringsInConfigForDeinterpolation(serviceAccountKey)
|
||||
}
|
||||
}
|
||||
|
||||
val sqlGenerator = BigQuerySqlGenerator(projectId, datasetLocation)
|
||||
val parsedCatalog =
|
||||
parseCatalog(
|
||||
sqlGenerator,
|
||||
defaultNamespace,
|
||||
rawNamespaceOverride.orElse(JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE),
|
||||
catalog,
|
||||
)
|
||||
val destinationHandler = BigQueryDestinationHandler(bigquery, datasetLocation)
|
||||
|
||||
val migrations =
|
||||
listOf(
|
||||
BigQueryDV2Migration(sqlGenerator, bigquery),
|
||||
BigqueryAirbyteMetaAndGenerationIdMigration(bigquery),
|
||||
)
|
||||
|
||||
if (uploadingMethod == UploadingMethod.STANDARD) {
|
||||
val bigQueryClientChunkSize = BigQueryUtils.getBigQueryClientChunkSize(config)
|
||||
val bigQueryLoadingStorageOperation =
|
||||
BigQueryDirectLoadingStorageOperation(
|
||||
bigquery,
|
||||
bigQueryClientChunkSize,
|
||||
BigQueryRecordFormatter(),
|
||||
sqlGenerator,
|
||||
destinationHandler,
|
||||
datasetLocation,
|
||||
)
|
||||
val syncOperation =
|
||||
DefaultSyncOperation<BigQueryDestinationState>(
|
||||
parsedCatalog,
|
||||
destinationHandler,
|
||||
defaultNamespace,
|
||||
{ initialStatus: DestinationInitialStatus<BigQueryDestinationState>, disableTD
|
||||
->
|
||||
StandardStreamOperation(
|
||||
bigQueryLoadingStorageOperation,
|
||||
initialStatus,
|
||||
disableTD
|
||||
)
|
||||
},
|
||||
migrations,
|
||||
disableTypeDedupe,
|
||||
)
|
||||
return createDirectUploadConsumer(
|
||||
outputRecordCollector,
|
||||
syncOperation,
|
||||
catalog,
|
||||
defaultNamespace,
|
||||
)
|
||||
}
|
||||
|
||||
val gcsNameTransformer = GcsNameTransformer()
|
||||
val gcsConfig = BigQueryUtils.getGcsCsvDestinationConfig(config)
|
||||
val keepStagingFiles = BigQueryUtils.isKeepFilesInGcs(config)
|
||||
val gcsOperations =
|
||||
GcsStorageOperations(gcsNameTransformer, gcsConfig.getS3Client(), gcsConfig)
|
||||
|
||||
val bigQueryGcsStorageOperations =
|
||||
BigQueryGcsStorageOperation(
|
||||
gcsOperations,
|
||||
gcsConfig,
|
||||
gcsNameTransformer,
|
||||
keepStagingFiles,
|
||||
bigquery,
|
||||
sqlGenerator,
|
||||
destinationHandler,
|
||||
)
|
||||
val syncOperation: SyncOperation =
|
||||
DefaultSyncOperation<BigQueryDestinationState>(
|
||||
parsedCatalog,
|
||||
destinationHandler,
|
||||
defaultNamespace,
|
||||
{ initialStatus: DestinationInitialStatus<BigQueryDestinationState>, disableTD ->
|
||||
StagingStreamOperations(
|
||||
bigQueryGcsStorageOperations,
|
||||
initialStatus,
|
||||
FileUploadFormat.CSV,
|
||||
V2_WITH_GENERATION,
|
||||
disableTD
|
||||
)
|
||||
},
|
||||
migrations,
|
||||
disableTypeDedupe,
|
||||
)
|
||||
return createStagingConsumer(
|
||||
outputRecordCollector,
|
||||
syncOperation,
|
||||
catalog,
|
||||
defaultNamespace,
|
||||
)
|
||||
}
|
||||
|
||||
private fun parseCatalog(
|
||||
sqlGenerator: BigQuerySqlGenerator,
|
||||
defaultNamespace: String,
|
||||
rawNamespaceOverride: String,
|
||||
catalog: ConfiguredAirbyteCatalog
|
||||
): ParsedCatalog {
|
||||
val catalogParser =
|
||||
CatalogParser(
|
||||
sqlGenerator,
|
||||
defaultNamespace = defaultNamespace,
|
||||
rawNamespace = rawNamespaceOverride,
|
||||
)
|
||||
|
||||
return catalogParser.parseCatalog(catalog)
|
||||
}
|
||||
|
||||
override val isV2Destination: Boolean
|
||||
get() = true
|
||||
|
||||
companion object {
|
||||
|
||||
private val REQUIRED_GCS_PERMISSIONS =
|
||||
listOf(
|
||||
"storage.multipartUploads.abort",
|
||||
"storage.multipartUploads.create",
|
||||
"storage.objects.create",
|
||||
"storage.objects.delete",
|
||||
"storage.objects.get",
|
||||
"storage.objects.list",
|
||||
)
|
||||
|
||||
@JvmStatic
|
||||
fun getBigQuery(config: JsonNode): BigQuery {
|
||||
val projectId = config[bqConstants.CONFIG_PROJECT_ID].asText()
|
||||
|
||||
try {
|
||||
val bigQueryBuilder = BigQueryOptions.newBuilder()
|
||||
val credentials = getServiceAccountCredentials(config)
|
||||
return bigQueryBuilder
|
||||
.setProjectId(projectId)
|
||||
.setCredentials(credentials)
|
||||
.setHeaderProvider(BigQueryUtils.headerProvider)
|
||||
.build()
|
||||
.service
|
||||
} catch (e: IOException) {
|
||||
throw RuntimeException(e)
|
||||
}
|
||||
}
|
||||
|
||||
@JvmStatic
|
||||
@Throws(IOException::class)
|
||||
fun getServiceAccountCredentials(config: JsonNode): GoogleCredentials {
|
||||
val serviceAccountKey = config[bqConstants.CONFIG_CREDS]
|
||||
// Follows this order of resolution:
|
||||
// https://cloud.google.com/java/docs/reference/google-auth-library/latest/com.google.auth.oauth2.GoogleCredentials#com_google_auth_oauth2_GoogleCredentials_getApplicationDefault
|
||||
if (serviceAccountKey == null) {
|
||||
log.info {
|
||||
"No service account key json is provided. It is required if you are using Airbyte cloud."
|
||||
}
|
||||
log.info { "Using the default service account credential from environment." }
|
||||
return GoogleCredentials.getApplicationDefault()
|
||||
}
|
||||
|
||||
// The JSON credential can either be a raw JSON object, or a serialized JSON object.
|
||||
val credentialsString =
|
||||
if (serviceAccountKey.isObject) serialize(serviceAccountKey)
|
||||
else serviceAccountKey.asText()
|
||||
return GoogleCredentials.fromStream(
|
||||
ByteArrayInputStream(credentialsString.toByteArray(Charsets.UTF_8)),
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
fun main(args: Array<String>) {
|
||||
AirbyteDestinationRunner.run(*args, additionalMicronautEnvs = additionalMicronautEnvs)
|
||||
addThrowableForDeinterpolation(BigQueryException::class.java)
|
||||
val destination: Destination = BigQueryDestination()
|
||||
log.info { "Starting Destination : ${destination.javaClass}" }
|
||||
IntegrationRunner(destination).run(args)
|
||||
log.info { "Completed Destination : ${destination.javaClass}" }
|
||||
}
|
||||
|
||||
@@ -3,18 +3,11 @@
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.load.data.Transformations
|
||||
import io.airbyte.cdk.integrations.destination.StandardNameTransformer
|
||||
|
||||
/**
|
||||
* A legacy class. This used to inherit from the old CDK's StandardNameTransformer. You probably
|
||||
* should avoid adding new uses of this class.
|
||||
*
|
||||
* (I have no explanation for the method names.)
|
||||
*/
|
||||
class BigQuerySQLNameTransformer {
|
||||
/** This seemingly is what we use for any table/column name. */
|
||||
fun convertStreamName(input: String): String {
|
||||
val result = Transformations.toAlphanumericAndUnderscore(input)
|
||||
class BigQuerySQLNameTransformer : StandardNameTransformer() {
|
||||
override fun convertStreamName(input: String): String {
|
||||
val result = super.convertStreamName(input)
|
||||
if (!result.substring(0, 1).matches("[A-Za-z_]".toRegex())) {
|
||||
// has to start with a letter or _
|
||||
return "_$result"
|
||||
@@ -29,8 +22,8 @@ class BigQuerySQLNameTransformer {
|
||||
* normalization. Reference: https://cloud.google.com/bigquery/docs/datasets#dataset-naming
|
||||
* </hidden-dataset>
|
||||
*/
|
||||
fun getNamespace(namespace: String): String {
|
||||
val normalizedName = Transformations.toAlphanumericAndUnderscore(namespace)
|
||||
override fun getNamespace(namespace: String): String {
|
||||
val normalizedName = super.convertStreamName(namespace)
|
||||
if (!normalizedName.substring(0, 1).matches("[A-Za-z0-9]".toRegex())) {
|
||||
return BigQueryConsts.NAMESPACE_PREFIX + normalizedName
|
||||
}
|
||||
@@ -38,7 +31,7 @@ class BigQuerySQLNameTransformer {
|
||||
}
|
||||
|
||||
@Deprecated("")
|
||||
fun getTmpTableName(streamName: String, randomSuffix: String): String {
|
||||
override fun getTmpTableName(streamName: String, randomSuffix: String): String {
|
||||
return convertStreamName("_airbyte_tmp" + "_" + randomSuffix + "_" + streamName)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -9,12 +9,16 @@ import com.google.cloud.RetryOption
|
||||
import com.google.cloud.bigquery.*
|
||||
import com.google.common.collect.ImmutableList
|
||||
import com.google.common.collect.ImmutableMap
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.commons.json.Jsons.jsonNode
|
||||
import java.util.*
|
||||
import java.util.stream.Collectors
|
||||
import org.apache.commons.lang3.StringUtils
|
||||
import org.apache.commons.lang3.tuple.ImmutablePair
|
||||
import org.apache.logging.log4j.util.Strings
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
import org.threeten.bp.Duration
|
||||
@@ -52,6 +56,7 @@ object BigQueryUtils {
|
||||
fun waitForQuery(queryJob: Job): Job? {
|
||||
try {
|
||||
val job = queryJob.waitFor()
|
||||
AirbyteExceptionHandler.addStringForDeinterpolation(job.etag)
|
||||
return job
|
||||
} catch (e: Exception) {
|
||||
LOGGER.error("Failed to wait for a query job:$queryJob")
|
||||
@@ -89,12 +94,12 @@ object BigQueryUtils {
|
||||
try {
|
||||
val partitioning =
|
||||
TimePartitioning.newBuilder(TimePartitioning.Type.DAY)
|
||||
.setField(Meta.COLUMN_NAME_AB_EXTRACTED_AT)
|
||||
.setField(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT)
|
||||
.build()
|
||||
|
||||
val clustering =
|
||||
Clustering.newBuilder()
|
||||
.setFields(ImmutableList.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT))
|
||||
.setFields(ImmutableList.of(JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT))
|
||||
.build()
|
||||
|
||||
val tableDefinition =
|
||||
@@ -121,7 +126,7 @@ object BigQueryUtils {
|
||||
@JvmStatic
|
||||
fun getGcsJsonNodeConfig(config: JsonNode): JsonNode {
|
||||
val loadingMethod = config[BigQueryConsts.LOADING_METHOD]
|
||||
return Jsons.valueToTree(
|
||||
return jsonNode(
|
||||
ImmutableMap.builder<Any, Any>()
|
||||
.put(BigQueryConsts.GCS_BUCKET_NAME, loadingMethod[BigQueryConsts.GCS_BUCKET_NAME])
|
||||
.put(BigQueryConsts.GCS_BUCKET_PATH, loadingMethod[BigQueryConsts.GCS_BUCKET_PATH])
|
||||
@@ -129,17 +134,19 @@ object BigQueryUtils {
|
||||
.put(BigQueryConsts.CREDENTIAL, loadingMethod[BigQueryConsts.CREDENTIAL])
|
||||
.put(
|
||||
BigQueryConsts.FORMAT,
|
||||
Jsons.readTree(
|
||||
"""{
|
||||
"format_type": "CSV",
|
||||
"flattening": "No flattening"
|
||||
}""".trimIndent()
|
||||
)
|
||||
deserialize("""{
|
||||
"format_type": "CSV",
|
||||
"flattening": "No flattening"
|
||||
}""")
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
fun getGcsCsvDestinationConfig(config: JsonNode): GcsDestinationConfig {
|
||||
return GcsDestinationConfig.getGcsDestinationConfig(getGcsJsonNodeConfig(config))
|
||||
}
|
||||
|
||||
/** @return a default schema name based on the config. */
|
||||
@JvmStatic
|
||||
fun getDatasetId(config: JsonNode): String {
|
||||
@@ -171,6 +178,32 @@ object BigQueryUtils {
|
||||
}
|
||||
}
|
||||
|
||||
fun getDisableTypeDedupFlag(config: JsonNode): Boolean {
|
||||
if (config.has(BigQueryConsts.DISABLE_TYPE_DEDUPE)) {
|
||||
return config[BigQueryConsts.DISABLE_TYPE_DEDUPE].asBoolean(false)
|
||||
}
|
||||
|
||||
return false
|
||||
}
|
||||
|
||||
// https://googleapis.dev/python/bigquery/latest/generated/google.cloud.bigquery.client.Client.html
|
||||
fun getBigQueryClientChunkSize(config: JsonNode): Int? {
|
||||
var chunkSizeFromConfig: Int? = null
|
||||
if (config.has(BigQueryConsts.BIG_QUERY_CLIENT_CHUNK_SIZE)) {
|
||||
chunkSizeFromConfig = config[BigQueryConsts.BIG_QUERY_CLIENT_CHUNK_SIZE].asInt()
|
||||
if (chunkSizeFromConfig <= 0) {
|
||||
LOGGER.error(
|
||||
"BigQuery client Chunk (buffer) size must be a positive number (MB), but was:$chunkSizeFromConfig"
|
||||
)
|
||||
throw IllegalArgumentException(
|
||||
"BigQuery client Chunk (buffer) size must be a positive number (MB)"
|
||||
)
|
||||
}
|
||||
chunkSizeFromConfig = chunkSizeFromConfig * BigQueryConsts.MiB
|
||||
}
|
||||
return chunkSizeFromConfig
|
||||
}
|
||||
|
||||
@JvmStatic
|
||||
fun getLoadingMethod(config: JsonNode): UploadingMethod {
|
||||
val loadingMethod = config[BigQueryConsts.LOADING_METHOD]
|
||||
@@ -186,9 +219,26 @@ object BigQueryUtils {
|
||||
}
|
||||
}
|
||||
|
||||
fun isKeepFilesInGcs(config: JsonNode): Boolean {
|
||||
val loadingMethod = config[BigQueryConsts.LOADING_METHOD]
|
||||
if (
|
||||
loadingMethod != null &&
|
||||
loadingMethod[BigQueryConsts.KEEP_GCS_FILES] != null &&
|
||||
(BigQueryConsts.KEEP_GCS_FILES_VAL ==
|
||||
loadingMethod[BigQueryConsts.KEEP_GCS_FILES].asText())
|
||||
) {
|
||||
LOGGER.info("All tmp files GCS will be kept in bucket when replication is finished")
|
||||
return true
|
||||
} else {
|
||||
LOGGER.info("All tmp files will be removed from GCS when replication is finished")
|
||||
return false
|
||||
}
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
fun waitForJobFinish(job: Job?) {
|
||||
if (job != null) {
|
||||
AirbyteExceptionHandler.addStringForDeinterpolation(job.etag)
|
||||
try {
|
||||
LOGGER.info("Waiting for Job {} to finish. Status: {}", job.jobId, job.status)
|
||||
// Default totalTimeout is 12 Hours, 30 minutes seems reasonable
|
||||
@@ -252,6 +302,6 @@ object BigQueryUtils {
|
||||
private val connectorNameOrDefault: String
|
||||
get() =
|
||||
Optional.ofNullable(System.getenv("WORKER_CONNECTOR_IMAGE"))
|
||||
.map { name: String -> name.replace("airbyte/", "").replace(":", "/") }
|
||||
.map { name: String -> name.replace("airbyte/", Strings.EMPTY).replace(":", "/") }
|
||||
.orElse("destination-bigquery")
|
||||
}
|
||||
|
||||
@@ -1,114 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.google.auth.oauth2.GoogleCredentials
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
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.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.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.github.oshai.kotlinlogging.KotlinLogging
|
||||
import io.micronaut.context.annotation.Factory
|
||||
import io.micronaut.context.annotation.Requires
|
||||
import jakarta.inject.Named
|
||||
import jakarta.inject.Singleton
|
||||
import java.io.ByteArrayInputStream
|
||||
import java.io.InputStream
|
||||
import java.nio.charset.StandardCharsets
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
@Factory
|
||||
class BigqueryBeansFactory {
|
||||
@Singleton fun getConfig(config: DestinationConfiguration) = config as BigqueryConfiguration
|
||||
|
||||
@Singleton
|
||||
@Requires(condition = BigqueryConfiguredForBulkLoad::class)
|
||||
fun getBulkLoadConfig(config: BigqueryConfiguration) = BigqueryBulkLoadConfiguration(config)
|
||||
|
||||
@Singleton
|
||||
@Named("checkNamespace")
|
||||
fun getCheckNamespace(config: BigqueryConfiguration): String = config.datasetId
|
||||
|
||||
@Singleton
|
||||
fun getChecker(
|
||||
catalog: DestinationCatalog,
|
||||
@Named("inputStream") stdinPipe: InputStream,
|
||||
taskLauncher: DestinationTaskLauncher,
|
||||
syncManager: SyncManager,
|
||||
) =
|
||||
DestinationCheckerSync(
|
||||
catalog,
|
||||
stdinPipe,
|
||||
WriteOperation(taskLauncher, syncManager),
|
||||
BigqueryCheckCleaner(),
|
||||
)
|
||||
|
||||
@Singleton
|
||||
fun getWriter(
|
||||
bigquery: BigQuery,
|
||||
config: BigqueryConfiguration,
|
||||
names: TableCatalog,
|
||||
streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
): TypingDedupingWriter {
|
||||
val destinationHandler = BigQueryDatabaseHandler(bigquery, config.datasetLocation.region)
|
||||
return TypingDedupingWriter(
|
||||
names,
|
||||
BigqueryDatabaseInitialStatusGatherer(bigquery),
|
||||
destinationHandler,
|
||||
BigqueryRawTableOperations(bigquery),
|
||||
TypingDedupingFinalTableOperations(
|
||||
BigQuerySqlGenerator(config.projectId, config.datasetLocation.region),
|
||||
destinationHandler,
|
||||
),
|
||||
disableTypeDedupe = config.disableTypingDeduping,
|
||||
streamStateStore,
|
||||
)
|
||||
}
|
||||
|
||||
@Singleton
|
||||
fun getBigqueryClient(config: BigqueryConfiguration): BigQuery {
|
||||
// Follows this order of resolution:
|
||||
// https://cloud.google.com/java/docs/reference/google-auth-library/latest/com.google.auth.oauth2.GoogleCredentials#com_google_auth_oauth2_GoogleCredentials_getApplicationDefault
|
||||
val credentials =
|
||||
if (config.credentialsJson == null) {
|
||||
logger.info {
|
||||
"No service account key json is provided. It is required if you are using Airbyte cloud."
|
||||
}
|
||||
logger.info { "Using the default service account credential from environment." }
|
||||
GoogleCredentials.getApplicationDefault()
|
||||
} else {
|
||||
// The JSON credential can either be a raw JSON object, or a serialized JSON object.
|
||||
GoogleCredentials.fromStream(
|
||||
ByteArrayInputStream(
|
||||
config.credentialsJson.toByteArray(StandardCharsets.UTF_8)
|
||||
),
|
||||
)
|
||||
}
|
||||
return BigQueryOptions.newBuilder()
|
||||
.setProjectId(config.projectId)
|
||||
.setCredentials(credentials)
|
||||
.setHeaderProvider(BigQueryUtils.headerProvider)
|
||||
.build()
|
||||
.service
|
||||
}
|
||||
}
|
||||
@@ -1,27 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.check
|
||||
|
||||
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
|
||||
|
||||
class BigqueryCheckCleaner : CheckCleaner<BigqueryConfiguration> {
|
||||
override fun cleanup(config: BigqueryConfiguration, stream: DestinationStream) {
|
||||
val bq = BigqueryBeansFactory().getBigqueryClient(config)
|
||||
bq.getTable(
|
||||
BigqueryRawTableNameGenerator(config).getTableName(stream.descriptor).toTableId()
|
||||
)
|
||||
?.delete()
|
||||
bq.getTable(
|
||||
BigqueryFinalTableNameGenerator(config).getTableName(stream.descriptor).toTableId()
|
||||
)
|
||||
?.delete()
|
||||
}
|
||||
}
|
||||
@@ -3,15 +3,21 @@
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.formatter
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.node.NullNode
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode
|
||||
import com.google.cloud.bigquery.Field
|
||||
import com.google.cloud.bigquery.QueryParameterValue
|
||||
import com.google.cloud.bigquery.Schema
|
||||
import com.google.cloud.bigquery.StandardSQLTypeName
|
||||
import io.airbyte.cdk.load.data.IntegerValue
|
||||
import io.airbyte.cdk.load.data.StringValue
|
||||
import io.airbyte.cdk.load.message.DestinationRecordRaw
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.util.serializeToString
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage
|
||||
import io.airbyte.commons.json.Jsons
|
||||
import io.airbyte.commons.json.Jsons.emptyObject
|
||||
import io.airbyte.commons.json.Jsons.serialize
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta
|
||||
import java.util.*
|
||||
import java.util.concurrent.TimeUnit
|
||||
|
||||
/**
|
||||
@@ -19,71 +25,44 @@ import java.util.concurrent.TimeUnit
|
||||
* corresponding uploader.
|
||||
*/
|
||||
class BigQueryRecordFormatter {
|
||||
|
||||
fun formatRecord(record: DestinationRecordRaw): String {
|
||||
val enrichedRecord = record.asEnrichedDestinationRecordAirbyteValue()
|
||||
|
||||
val outputRecord = mutableMapOf<String, Any?>()
|
||||
enrichedRecord.airbyteMetaFields.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["sync_id"] = record.stream.syncId
|
||||
outputRecord[key] = protocolMeta.serializeToString()
|
||||
// TODO we should do this for direct-load tables
|
||||
// val serializedAirbyteMeta = (value.abValue as
|
||||
// ObjectValue).serializeToString()
|
||||
// outputRecord[key] = serializedAirbyteMeta
|
||||
}
|
||||
Meta.COLUMN_NAME_AB_RAW_ID ->
|
||||
outputRecord[key] = (value.abValue as StringValue).value
|
||||
Meta.COLUMN_NAME_AB_GENERATION_ID ->
|
||||
outputRecord[key] = (value.abValue as IntegerValue).value
|
||||
}
|
||||
}
|
||||
|
||||
outputRecord[Meta.COLUMN_NAME_DATA] = record.asJsonRecord().serializeToString()
|
||||
|
||||
return outputRecord.serializeToString()
|
||||
fun formatRecord(recordMessage: PartialAirbyteMessage, generationId: Long): String {
|
||||
val record = emptyObject() as ObjectNode
|
||||
record.put(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, UUID.randomUUID().toString())
|
||||
record.put(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT,
|
||||
getEmittedAtField(recordMessage.record)
|
||||
)
|
||||
record.set<JsonNode>(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, NullNode.instance)
|
||||
record.put(JavaBaseConstants.COLUMN_NAME_DATA, recordMessage.serialized)
|
||||
record.put(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_META,
|
||||
Jsons.serialize<AirbyteRecordMessageMeta>(recordMessage.record!!.meta!!)
|
||||
)
|
||||
record.put(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, generationId)
|
||||
return serialize(record)
|
||||
}
|
||||
|
||||
private fun getExtractedAt(extractedAtMillis: Long): String? {
|
||||
private fun getEmittedAtField(recordMessage: PartialAirbyteRecordMessage?): String? {
|
||||
// Bigquery represents TIMESTAMP to the microsecond precision, so we convert to microseconds
|
||||
// then use BQ helpers to string-format correctly.
|
||||
// then
|
||||
// use BQ helpers to string-format correctly.
|
||||
val emittedAtMicroseconds =
|
||||
TimeUnit.MICROSECONDS.convert(extractedAtMillis, TimeUnit.MILLISECONDS)
|
||||
TimeUnit.MICROSECONDS.convert(recordMessage!!.emittedAt, TimeUnit.MILLISECONDS)
|
||||
return QueryParameterValue.timestamp(emittedAtMicroseconds).value
|
||||
}
|
||||
|
||||
companion object {
|
||||
// This is the schema used to represent the final raw table
|
||||
val SCHEMA_V2: Schema =
|
||||
Schema.of(
|
||||
Field.of(Meta.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING),
|
||||
Field.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP),
|
||||
Field.of(Meta.COLUMN_NAME_AB_LOADED_AT, StandardSQLTypeName.TIMESTAMP),
|
||||
Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING),
|
||||
Field.of(Meta.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING),
|
||||
Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64)
|
||||
)
|
||||
|
||||
// This schema defines the CSV format used for the load job. It differs from SCHEMA_V2 by
|
||||
// omitting the COLUMN_NAME_AB_LOADED_AT field and by rearranging the column order.
|
||||
val CSV_SCHEMA: Schema =
|
||||
Schema.of(
|
||||
Field.of(Meta.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING),
|
||||
Field.of(Meta.COLUMN_NAME_AB_EXTRACTED_AT, StandardSQLTypeName.TIMESTAMP),
|
||||
Field.of(Meta.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING),
|
||||
Field.of(Meta.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64),
|
||||
Field.of(Meta.COLUMN_NAME_DATA, StandardSQLTypeName.STRING),
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_RAW_ID, StandardSQLTypeName.STRING),
|
||||
Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT,
|
||||
StandardSQLTypeName.TIMESTAMP
|
||||
),
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT, StandardSQLTypeName.TIMESTAMP),
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_DATA, StandardSQLTypeName.STRING),
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING),
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,33 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.migrators
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationHandler
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.migrators.Migration
|
||||
import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryV1V2Migrator
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import kotlin.math.log
|
||||
|
||||
class BigQueryDV2Migration(private val sqlGenerator: BigQuerySqlGenerator, bigQuery: BigQuery) :
|
||||
Migration<BigQueryDestinationState> {
|
||||
private val log = KotlinLogging.logger {}
|
||||
private val legacyV1V2migrator = BigQueryV1V2Migrator(bigQuery, BigQuerySQLNameTransformer())
|
||||
override fun migrateIfNecessary(
|
||||
destinationHandler: DestinationHandler<BigQueryDestinationState>,
|
||||
stream: StreamConfig,
|
||||
state: DestinationInitialStatus<BigQueryDestinationState>
|
||||
): Migration.MigrationResult<BigQueryDestinationState> {
|
||||
log.info { "Initializing DV2 Migration check" }
|
||||
legacyV1V2migrator.migrateIfNecessary(sqlGenerator, destinationHandler, stream)
|
||||
// Invalidate state because rawTableExists could be false but we don't use it yet for
|
||||
// anything ?
|
||||
return Migration.MigrationResult(BigQueryDestinationState(false), true)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,18 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.migrators
|
||||
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.migrators.MinimumDestinationState
|
||||
|
||||
data class BigQueryDestinationState(private val needsSoftReset: Boolean) : MinimumDestinationState {
|
||||
override fun needsSoftReset(): Boolean {
|
||||
return needsSoftReset
|
||||
}
|
||||
|
||||
@Suppress("UNCHECKED_CAST")
|
||||
override fun <T : MinimumDestinationState> withSoftReset(needsSoftReset: Boolean): T {
|
||||
return copy(needsSoftReset = needsSoftReset) as T
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,97 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.migrators
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.Field
|
||||
import com.google.cloud.bigquery.Schema
|
||||
import com.google.cloud.bigquery.StandardSQLTypeName
|
||||
import com.google.cloud.bigquery.StandardTableDefinition
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationHandler
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.DestinationInitialStatus
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.migrators.Migration
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
|
||||
class BigqueryAirbyteMetaAndGenerationIdMigration(private val bigquery: BigQuery) :
|
||||
Migration<BigQueryDestinationState> {
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
override fun migrateIfNecessary(
|
||||
destinationHandler: DestinationHandler<BigQueryDestinationState>,
|
||||
stream: StreamConfig,
|
||||
state: DestinationInitialStatus<BigQueryDestinationState>
|
||||
): Migration.MigrationResult<BigQueryDestinationState> {
|
||||
if (!state.initialRawTableStatus.rawTableExists) {
|
||||
// The raw table doesn't exist. No migration necessary. Update the state.
|
||||
logger.info {
|
||||
"Skipping airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because the raw table doesn't exist"
|
||||
}
|
||||
return Migration.MigrationResult(state.destinationState, false)
|
||||
}
|
||||
|
||||
val rawTable = bigquery.getTable(TableId.of(stream.id.rawNamespace, stream.id.rawName))
|
||||
// if the schema is null, then we have bigger problems
|
||||
val rawFields = rawTable.getDefinition<StandardTableDefinition>().schema!!.fields
|
||||
val hasMeta = rawFields.any { it.name == JavaBaseConstants.COLUMN_NAME_AB_META }
|
||||
if (hasMeta) {
|
||||
// We've already executed the migration. Do nothing here.
|
||||
logger.info {
|
||||
"Skipping airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName} because the table already has the columns"
|
||||
}
|
||||
return Migration.MigrationResult(state.destinationState, false)
|
||||
}
|
||||
|
||||
logger.info {
|
||||
"Executing airbyte_meta/generation_id migration for ${stream.id.originalNamespace}.${stream.id.originalName}"
|
||||
}
|
||||
|
||||
// Define the new columns we're adding.
|
||||
// Add meta to the raw table
|
||||
val rawMetaField =
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_META, StandardSQLTypeName.STRING)
|
||||
// And add generation ID to raw+final tables.
|
||||
val generationIdField =
|
||||
Field.of(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID, StandardSQLTypeName.INT64)
|
||||
|
||||
// The way bigquery does an "alter table" is by just setting the entire table definition
|
||||
// (unless you want to write actual SQL, of course).
|
||||
// This adds the column as NULLABLE.
|
||||
val newRawTable =
|
||||
rawTable
|
||||
.toBuilder()
|
||||
.setDefinition(
|
||||
StandardTableDefinition.of(
|
||||
Schema.of(rawFields + rawMetaField + generationIdField)
|
||||
)
|
||||
)
|
||||
.build()
|
||||
newRawTable.update()
|
||||
|
||||
if (state.isFinalTablePresent) {
|
||||
val finalTable =
|
||||
bigquery.getTable(TableId.of(stream.id.finalNamespace, stream.id.finalName))
|
||||
val finalFields = finalTable.getDefinition<StandardTableDefinition>().schema!!.fields
|
||||
val airbyteMetaIndex =
|
||||
finalFields.indexOfFirst { it.name == JavaBaseConstants.COLUMN_NAME_AB_META }
|
||||
// Insert generation_id immediately after airbyte_meta
|
||||
val newFinalFields =
|
||||
finalFields.subList(0, airbyteMetaIndex + 1) +
|
||||
generationIdField +
|
||||
finalFields.subList(airbyteMetaIndex + 1, finalFields.size)
|
||||
val newFinalTable =
|
||||
finalTable
|
||||
.toBuilder()
|
||||
.setDefinition(StandardTableDefinition.of(Schema.of(newFinalFields)))
|
||||
.build()
|
||||
newFinalTable.update()
|
||||
}
|
||||
|
||||
// We need to refetch the initial state, because we modified the final table schema.
|
||||
return Migration.MigrationResult(state.destinationState, true)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,117 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.operation
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.FormatOptions
|
||||
import com.google.cloud.bigquery.JobId
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.TableDataWriteChannel
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import com.google.cloud.bigquery.WriteChannelConfiguration
|
||||
import com.google.common.util.concurrent.RateLimiter
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.nio.ByteBuffer
|
||||
import java.nio.charset.StandardCharsets
|
||||
import java.util.stream.Stream
|
||||
|
||||
private val log = KotlinLogging.logger {}
|
||||
|
||||
class BigQueryDirectLoadingStorageOperation(
|
||||
bigquery: BigQuery,
|
||||
private val bigQueryClientChunkSize: Int?,
|
||||
private val bigQueryRecordFormatter: BigQueryRecordFormatter,
|
||||
sqlGenerator: BigQuerySqlGenerator,
|
||||
destinationHandler: BigQueryDestinationHandler,
|
||||
datasetLocation: String
|
||||
) :
|
||||
BigQueryStorageOperation<Stream<PartialAirbyteMessage>>(
|
||||
bigquery,
|
||||
sqlGenerator,
|
||||
destinationHandler,
|
||||
datasetLocation,
|
||||
) {
|
||||
private val rateLimiter: RateLimiter = RateLimiter.create(0.07)
|
||||
companion object {
|
||||
private const val HTTP_STATUS_CODE_FORBIDDEN = 403
|
||||
private const val HTTP_STATUS_CODE_NOT_FOUND = 404
|
||||
|
||||
private val CONFIG_ERROR_MSG =
|
||||
"""
|
||||
|Failed to write to destination schema.
|
||||
| 1. Make sure you have all required permissions for writing to the schema.
|
||||
| 2. Make sure that the actual destination schema's location corresponds to location provided in connector's config.
|
||||
| 3. Try to change the "Destination schema" from "Mirror Source Structure" (if it's set) tp the "Destination Default" option.
|
||||
|More details:
|
||||
|""".trimMargin()
|
||||
}
|
||||
override fun writeToStage(
|
||||
streamConfig: StreamConfig,
|
||||
suffix: String,
|
||||
data: Stream<PartialAirbyteMessage>
|
||||
) {
|
||||
// TODO: why do we need ratelimiter, and using unstable API from Google's guava
|
||||
rateLimiter.acquire()
|
||||
val tableId = tableId(streamConfig.id, suffix)
|
||||
log.info {
|
||||
"Writing data to table $tableId with schema ${BigQueryRecordFormatter.SCHEMA_V2}"
|
||||
}
|
||||
val writeChannel = initWriteChannel(tableId)
|
||||
writeChannel.use {
|
||||
data.forEach { record ->
|
||||
val byteArray =
|
||||
"${bigQueryRecordFormatter.formatRecord(record, streamConfig.generationId)} ${System.lineSeparator()}".toByteArray(
|
||||
StandardCharsets.UTF_8,
|
||||
)
|
||||
it.write(ByteBuffer.wrap(byteArray))
|
||||
}
|
||||
}
|
||||
log.info { "Writing to channel completed for $tableId" }
|
||||
val job = writeChannel.job
|
||||
BigQueryUtils.waitForJobFinish(job)
|
||||
}
|
||||
|
||||
private fun initWriteChannel(tableId: TableId): TableDataWriteChannel {
|
||||
val writeChannelConfiguration =
|
||||
WriteChannelConfiguration.newBuilder(tableId)
|
||||
.setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED)
|
||||
.setSchema(BigQueryRecordFormatter.SCHEMA_V2)
|
||||
.setFormatOptions(FormatOptions.json())
|
||||
.build() // new-line delimited json.
|
||||
|
||||
val job =
|
||||
JobId.newBuilder()
|
||||
.setRandomJob()
|
||||
.setLocation(datasetLocation)
|
||||
.setProject(bigquery.options.projectId)
|
||||
.build()
|
||||
|
||||
val writer: TableDataWriteChannel
|
||||
|
||||
try {
|
||||
writer = bigquery.writer(job, writeChannelConfiguration)
|
||||
} catch (e: BigQueryException) {
|
||||
if (e.code == HTTP_STATUS_CODE_FORBIDDEN || e.code == HTTP_STATUS_CODE_NOT_FOUND) {
|
||||
throw ConfigErrorException(CONFIG_ERROR_MSG + e)
|
||||
} else {
|
||||
throw BigQueryException(e.code, e.message)
|
||||
}
|
||||
}
|
||||
|
||||
// this this optional value. If not set - use default client's value (15MiG)
|
||||
if (bigQueryClientChunkSize != null) {
|
||||
writer.setChunkSize(bigQueryClientChunkSize)
|
||||
}
|
||||
return writer
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,158 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.operation
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.FormatOptions
|
||||
import com.google.cloud.bigquery.Job
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.LoadJobConfiguration
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsNameTransformer
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsStorageOperations
|
||||
import io.airbyte.cdk.integrations.destination.record_buffer.SerializableBuffer
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamId
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.util.*
|
||||
import org.joda.time.DateTime
|
||||
import org.joda.time.DateTimeZone
|
||||
|
||||
private val log = KotlinLogging.logger {}
|
||||
|
||||
class BigQueryGcsStorageOperation(
|
||||
private val gcsStorageOperations: GcsStorageOperations,
|
||||
private val gcsConfig: GcsDestinationConfig,
|
||||
private val gcsNameTransformer: GcsNameTransformer,
|
||||
private val keepStagingFiles: Boolean,
|
||||
bigquery: BigQuery,
|
||||
sqlGenerator: BigQuerySqlGenerator,
|
||||
destinationHandler: BigQueryDestinationHandler
|
||||
) :
|
||||
BigQueryStorageOperation<SerializableBuffer>(
|
||||
bigquery,
|
||||
sqlGenerator,
|
||||
destinationHandler,
|
||||
datasetLocation = gcsConfig.bucketRegion!!
|
||||
) {
|
||||
private val connectionId = UUID.randomUUID()
|
||||
private val syncDateTime = DateTime.now(DateTimeZone.UTC)
|
||||
override fun prepareStage(streamId: StreamId, suffix: String, replace: Boolean) {
|
||||
super.prepareStage(streamId, suffix, replace)
|
||||
// prepare staging bucket
|
||||
// TODO should this also use the suffix?
|
||||
log.info { "Creating bucket ${gcsConfig.bucketName}" }
|
||||
gcsStorageOperations.createBucketIfNotExists()
|
||||
}
|
||||
|
||||
override fun cleanupStage(streamId: StreamId) {
|
||||
if (keepStagingFiles) return
|
||||
|
||||
val stagingRootPath = stagingRootPath(streamId)
|
||||
log.info { "Cleaning up staging path at $stagingRootPath" }
|
||||
gcsStorageOperations.dropBucketObject(stagingRootPath)
|
||||
}
|
||||
|
||||
override fun writeToStage(
|
||||
streamConfig: StreamConfig,
|
||||
suffix: String,
|
||||
data: SerializableBuffer
|
||||
) {
|
||||
val stagedFileName: String =
|
||||
uploadRecordsToStage(streamConfig.id, suffix, data, streamConfig.generationId)
|
||||
copyIntoTableFromStage(streamConfig.id, suffix, stagedFileName)
|
||||
}
|
||||
|
||||
private fun uploadRecordsToStage(
|
||||
streamId: StreamId,
|
||||
suffix: String,
|
||||
buffer: SerializableBuffer,
|
||||
generationId: Long,
|
||||
): String {
|
||||
val objectPath: String = stagingFullPath(streamId)
|
||||
log.info {
|
||||
"Uploading records to for ${streamId.rawNamespace}.${streamId.rawName}$suffix to path $objectPath"
|
||||
}
|
||||
return gcsStorageOperations.uploadRecordsToBucket(
|
||||
buffer,
|
||||
streamId.rawNamespace,
|
||||
objectPath,
|
||||
generationId
|
||||
)
|
||||
}
|
||||
|
||||
private fun copyIntoTableFromStage(streamId: StreamId, suffix: String, stagedFileName: String) {
|
||||
val tableId = tableId(streamId, suffix)
|
||||
val stagingPath = stagingFullPath(streamId)
|
||||
val fullFilePath = "gs://${gcsConfig.bucketName}/$stagingPath$stagedFileName"
|
||||
log.info { "Uploading records from file $fullFilePath to target Table $tableId" }
|
||||
val configuration =
|
||||
LoadJobConfiguration.builder(tableId, fullFilePath)
|
||||
.setFormatOptions(FormatOptions.csv())
|
||||
.setSchema(BigQueryRecordFormatter.SCHEMA_V2)
|
||||
.setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
|
||||
.setJobTimeoutMs(600000L) // 10 min
|
||||
.build()
|
||||
|
||||
val loadJob: Job = this.bigquery.create(JobInfo.of(configuration))
|
||||
log.info {
|
||||
"[${loadJob.jobId}] Created a new job to upload record(s) to target table $tableId: $loadJob"
|
||||
}
|
||||
try {
|
||||
BigQueryUtils.waitForJobFinish(loadJob)
|
||||
log.info {
|
||||
"[${loadJob.jobId}] Target table $tableId is successfully appended with staging files"
|
||||
}
|
||||
} catch (e: BigQueryException) {
|
||||
throw RuntimeException(
|
||||
String.format(
|
||||
"[%s] Failed to upload staging files to destination table %s",
|
||||
loadJob.jobId,
|
||||
tableId
|
||||
),
|
||||
e
|
||||
)
|
||||
} catch (e: InterruptedException) {
|
||||
throw RuntimeException(
|
||||
String.format(
|
||||
"[%s] Failed to upload staging files to destination table %s",
|
||||
loadJob.jobId,
|
||||
tableId
|
||||
),
|
||||
e
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
private fun stagingFullPath(streamId: StreamId): String {
|
||||
return gcsNameTransformer.applyDefaultCase(
|
||||
String.format(
|
||||
"%s%s/%02d/%02d/%02d/%s/",
|
||||
stagingRootPath(streamId),
|
||||
syncDateTime.year().get(),
|
||||
syncDateTime.monthOfYear().get(),
|
||||
syncDateTime.dayOfMonth().get(),
|
||||
syncDateTime.hourOfDay().get(),
|
||||
connectionId
|
||||
)
|
||||
)
|
||||
}
|
||||
|
||||
private fun stagingRootPath(streamId: StreamId): String {
|
||||
return gcsNameTransformer.applyDefaultCase(
|
||||
String.format(
|
||||
"%s/%s_%s/",
|
||||
gcsConfig.bucketPath,
|
||||
gcsNameTransformer.convertStreamName(streamId.rawNamespace),
|
||||
gcsNameTransformer.convertStreamName(streamId.rawName)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,177 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.operation
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import com.google.cloud.bigquery.TableResult
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.integrations.base.destination.operation.StorageOperation
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamId
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.TyperDeduperUtil
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.time.Instant
|
||||
import java.util.Optional
|
||||
import java.util.concurrent.ConcurrentHashMap
|
||||
|
||||
private val log = KotlinLogging.logger {}
|
||||
|
||||
abstract class BigQueryStorageOperation<Data>(
|
||||
protected val bigquery: BigQuery,
|
||||
private val sqlGenerator: BigQuerySqlGenerator,
|
||||
private val destinationHandler: BigQueryDestinationHandler,
|
||||
protected val datasetLocation: String
|
||||
) : StorageOperation<Data> {
|
||||
private val existingSchemas = ConcurrentHashMap.newKeySet<String>()
|
||||
override fun prepareStage(streamId: StreamId, suffix: String, replace: Boolean) {
|
||||
// Prepare staging table. For overwrite, it does drop-create so we can skip explicit create.
|
||||
if (replace) {
|
||||
truncateStagingTable(streamId, suffix)
|
||||
} else {
|
||||
createStagingTable(streamId, suffix)
|
||||
}
|
||||
}
|
||||
|
||||
override fun overwriteStage(streamId: StreamId, suffix: String) {
|
||||
if (suffix == "") {
|
||||
throw IllegalArgumentException("Cannot overwrite raw table with empty suffix")
|
||||
}
|
||||
bigquery.delete(tableId(streamId, ""))
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"""ALTER TABLE `${streamId.rawNamespace}`.`${streamId.rawName}$suffix` RENAME TO `${streamId.rawName}`"""
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
override fun transferFromTempStage(streamId: StreamId, suffix: String) {
|
||||
if (suffix == "") {
|
||||
throw IllegalArgumentException(
|
||||
"Cannot transfer records from temp raw table with empty suffix"
|
||||
)
|
||||
}
|
||||
// TODO figure out how to make this work
|
||||
// something about incompatible partitioning spec (probably b/c we're copying from a temp
|
||||
// table partitioned on generation ID into an old real raw table partitioned on
|
||||
// extracted_at)
|
||||
val tempRawTable = tableId(streamId, suffix)
|
||||
// val jobConf =
|
||||
// CopyJobConfiguration.newBuilder(tableId(streamId, ""), tempRawTable)
|
||||
// .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
|
||||
// .build()
|
||||
// val job = bigquery.create(JobInfo.of(jobConf))
|
||||
// BigQueryUtils.waitForJobFinish(job)
|
||||
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"""
|
||||
INSERT INTO `${streamId.rawNamespace}`.`${streamId.rawName}`
|
||||
SELECT * FROM `${streamId.rawNamespace}`.`${streamId.rawName}$suffix`
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
bigquery.delete(tempRawTable)
|
||||
}
|
||||
|
||||
override fun getStageGeneration(streamId: StreamId, suffix: String): Long? {
|
||||
val result: TableResult =
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT _airbyte_generation_id FROM ${streamId.rawNamespace}.${streamId.rawName}$suffix LIMIT 1"
|
||||
),
|
||||
)
|
||||
if (result.totalRows == 0L) {
|
||||
return null
|
||||
}
|
||||
val value = result.iterateAll().first().get(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID)
|
||||
return if (value == null || value.isNull) {
|
||||
0
|
||||
} else {
|
||||
value.longValue
|
||||
}
|
||||
}
|
||||
|
||||
private fun createStagingTable(streamId: StreamId, suffix: String) {
|
||||
BigQueryUtils.createPartitionedTableIfNotExists(
|
||||
bigquery,
|
||||
tableId(streamId, suffix),
|
||||
BigQueryRecordFormatter.SCHEMA_V2,
|
||||
)
|
||||
}
|
||||
|
||||
private fun dropStagingTable(streamId: StreamId, suffix: String) {
|
||||
bigquery.delete(tableId(streamId, suffix))
|
||||
}
|
||||
|
||||
/**
|
||||
* "Truncates" table, this is a workaround to the issue with TRUNCATE TABLE in BigQuery where
|
||||
* the table's partition filter must be turned off to truncate. Since deleting a table is a free
|
||||
* operation this option re-uses functions that already exist
|
||||
*/
|
||||
private fun truncateStagingTable(streamId: StreamId, suffix: String) {
|
||||
val tableId = TableId.of(streamId.rawNamespace, streamId.rawName)
|
||||
log.info { "Truncating raw table $tableId" }
|
||||
dropStagingTable(streamId, suffix)
|
||||
createStagingTable(streamId, suffix)
|
||||
}
|
||||
|
||||
override fun cleanupStage(streamId: StreamId) {
|
||||
log.info { "Nothing to cleanup in stage for Streaming inserts" }
|
||||
}
|
||||
|
||||
override fun createFinalTable(streamConfig: StreamConfig, suffix: String, replace: Boolean) {
|
||||
destinationHandler.execute(sqlGenerator.createTable(streamConfig, suffix, replace))
|
||||
}
|
||||
|
||||
override fun softResetFinalTable(streamConfig: StreamConfig) {
|
||||
TyperDeduperUtil.executeSoftReset(
|
||||
sqlGenerator = sqlGenerator,
|
||||
destinationHandler = destinationHandler,
|
||||
streamConfig,
|
||||
)
|
||||
}
|
||||
|
||||
override fun overwriteFinalTable(streamConfig: StreamConfig, tmpTableSuffix: String) {
|
||||
if (tmpTableSuffix == "") {
|
||||
throw IllegalArgumentException("Cannot overwrite final table with empty suffix")
|
||||
}
|
||||
log.info {
|
||||
"Overwriting table ${streamConfig.id.finalTableId(BigQuerySqlGenerator.QUOTE)} with ${
|
||||
streamConfig.id.finalTableId(
|
||||
BigQuerySqlGenerator.QUOTE,
|
||||
tmpTableSuffix,
|
||||
)
|
||||
}"
|
||||
}
|
||||
destinationHandler.execute(
|
||||
sqlGenerator.overwriteFinalTable(streamConfig.id, tmpTableSuffix),
|
||||
)
|
||||
}
|
||||
|
||||
override fun typeAndDedupe(
|
||||
streamConfig: StreamConfig,
|
||||
maxProcessedTimestamp: Optional<Instant>,
|
||||
finalTableSuffix: String
|
||||
) {
|
||||
TyperDeduperUtil.executeTypeAndDedupe(
|
||||
sqlGenerator = sqlGenerator,
|
||||
destinationHandler = destinationHandler,
|
||||
streamConfig,
|
||||
maxProcessedTimestamp,
|
||||
finalTableSuffix,
|
||||
)
|
||||
}
|
||||
|
||||
companion object {
|
||||
fun tableId(streamId: StreamId, suffix: String = ""): TableId =
|
||||
TableId.of(streamId.rawNamespace, streamId.rawName + suffix)
|
||||
}
|
||||
}
|
||||
@@ -1,67 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.spec
|
||||
|
||||
import io.airbyte.cdk.load.command.DestinationConfiguration
|
||||
import io.airbyte.cdk.load.command.DestinationConfigurationFactory
|
||||
import io.airbyte.cdk.load.command.gcs.GcsClientConfiguration
|
||||
import io.airbyte.cdk.load.write.db.DbConstants
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
data class BigqueryConfiguration(
|
||||
val projectId: String,
|
||||
val datasetLocation: BigqueryRegion,
|
||||
val datasetId: String,
|
||||
val loadingMethod: LoadingMethodConfiguration,
|
||||
val credentialsJson: String?,
|
||||
val transformationPriority: TransformationPriority,
|
||||
val rawTableDataset: String,
|
||||
val disableTypingDeduping: Boolean,
|
||||
) : DestinationConfiguration() {
|
||||
override val numOpenStreamWorkers = 3
|
||||
}
|
||||
|
||||
sealed interface LoadingMethodConfiguration
|
||||
|
||||
data object BatchedStandardInsertConfiguration : LoadingMethodConfiguration
|
||||
|
||||
data class GcsStagingConfiguration(
|
||||
val gcsClientConfig: GcsClientConfiguration,
|
||||
val filePostProcessing: GcsFilePostProcessing,
|
||||
) : LoadingMethodConfiguration
|
||||
|
||||
@Singleton
|
||||
class BigqueryConfigurationFactory :
|
||||
DestinationConfigurationFactory<BigquerySpecification, BigqueryConfiguration> {
|
||||
override fun makeWithoutExceptionHandling(pojo: BigquerySpecification): BigqueryConfiguration {
|
||||
val loadingMethodConfig =
|
||||
when (pojo.loadingMethod) {
|
||||
is GcsStagingSpecification -> {
|
||||
val gcsStagingSpec = pojo.loadingMethod as GcsStagingSpecification
|
||||
GcsStagingConfiguration(
|
||||
GcsClientConfiguration(gcsStagingSpec, pojo.datasetLocation.gcsRegion),
|
||||
gcsStagingSpec.filePostProcessing ?: GcsFilePostProcessing.DELETE,
|
||||
)
|
||||
}
|
||||
is BatchedStandardInsertSpecification,
|
||||
null -> BatchedStandardInsertConfiguration
|
||||
}
|
||||
return BigqueryConfiguration(
|
||||
projectId = pojo.projectId,
|
||||
pojo.datasetLocation,
|
||||
datasetId = pojo.datasetId,
|
||||
loadingMethodConfig,
|
||||
credentialsJson = pojo.credentialsJson,
|
||||
pojo.transformationPriority ?: TransformationPriority.INTERACTIVE,
|
||||
rawTableDataset =
|
||||
if (pojo.rawTableDataset.isNullOrBlank()) {
|
||||
DbConstants.DEFAULT_RAW_TABLE_NAMESPACE
|
||||
} else {
|
||||
pojo.rawTableDataset!!
|
||||
},
|
||||
disableTypingDeduping = pojo.disableTypingDeduping ?: false,
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -1,238 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.spec
|
||||
|
||||
import com.fasterxml.jackson.annotation.JsonProperty
|
||||
import com.fasterxml.jackson.annotation.JsonPropertyDescription
|
||||
import com.fasterxml.jackson.annotation.JsonSubTypes
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo
|
||||
import com.fasterxml.jackson.annotation.JsonValue
|
||||
import com.fasterxml.jackson.core.JsonParser
|
||||
import com.fasterxml.jackson.core.JsonProcessingException
|
||||
import com.fasterxml.jackson.databind.DeserializationContext
|
||||
import com.fasterxml.jackson.databind.JsonDeserializer
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.annotation.JsonDeserialize
|
||||
import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaDescription
|
||||
import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaInject
|
||||
import com.kjetland.jackson.jsonSchema.annotations.JsonSchemaTitle
|
||||
import io.airbyte.cdk.command.ConfigurationSpecification
|
||||
import io.airbyte.cdk.load.command.gcs.GcsAuthSpecification
|
||||
import io.airbyte.cdk.load.command.gcs.GcsCommonSpecification
|
||||
import io.airbyte.cdk.load.command.gcs.GcsHmacKeySpecification
|
||||
import io.airbyte.cdk.load.command.gcs.GcsRegion
|
||||
import io.airbyte.cdk.load.spec.DestinationSpecificationExtension
|
||||
import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode
|
||||
import jakarta.inject.Singleton
|
||||
import java.io.IOException
|
||||
|
||||
@Singleton
|
||||
class BigquerySpecification : ConfigurationSpecification() {
|
||||
@get:JsonSchemaTitle("Project ID")
|
||||
@get:JsonPropertyDescription(
|
||||
"""The GCP project ID for the project containing the target BigQuery dataset. Read more <a href="https://cloud.google.com/resource-manager/docs/creating-managing-projects#identifying_projects">here</a>.""",
|
||||
)
|
||||
@get:JsonProperty("project_id")
|
||||
@get:JsonSchemaInject(json = """{"group": "connection", "order": 0}""")
|
||||
val projectId: String = ""
|
||||
|
||||
@get:JsonSchemaTitle("Dataset Location")
|
||||
@get:JsonPropertyDescription(
|
||||
"""The location of the dataset. Warning: Changes made after creation will not be applied. Read more <a href="https://cloud.google.com/bigquery/docs/locations">here</a>.""",
|
||||
)
|
||||
@get:JsonProperty("dataset_location")
|
||||
@get:JsonSchemaInject(json = """{"group": "connection", "order": 1}""")
|
||||
val datasetLocation: BigqueryRegion = BigqueryRegion.US_EAST1
|
||||
|
||||
@get:JsonSchemaTitle("Default Dataset ID")
|
||||
@get:JsonPropertyDescription(
|
||||
"""The default BigQuery Dataset ID that tables are replicated to if the source does not specify a namespace. Read more <a href="https://cloud.google.com/bigquery/docs/datasets#create-dataset">here</a>.""",
|
||||
)
|
||||
@get:JsonProperty("dataset_id")
|
||||
@get:JsonSchemaInject(json = """{"group": "connection", "order": 2}""")
|
||||
val datasetId: String = ""
|
||||
|
||||
@get:JsonSchemaTitle("Loading Method")
|
||||
@get:JsonPropertyDescription("""The way data will be uploaded to BigQuery.""")
|
||||
@get:JsonProperty("loading_method")
|
||||
@get:JsonSchemaInject(json = """{"group": "connection", "order": 3, "display_type": "radio"}""")
|
||||
val loadingMethod: LoadingMethodSpecification? = BatchedStandardInsertSpecification()
|
||||
|
||||
// older versions of the connector represented this field as an actual JSON object,
|
||||
// so we need to use the RawJsonDeserializer.
|
||||
@get:JsonDeserialize(using = RawJsonDeserializer::class)
|
||||
@get:JsonSchemaTitle("Service Account Key JSON (Required for cloud, optional for open-source)")
|
||||
@get:JsonPropertyDescription(
|
||||
"""The contents of the JSON service account key. Check out the <a href="https://docs.airbyte.com/integrations/destinations/bigquery#service-account-key">docs</a> if you need help generating this key. Default credentials will be used if this field is left empty.""",
|
||||
)
|
||||
@get:JsonProperty("credentials_json")
|
||||
@get:JsonSchemaInject(
|
||||
json =
|
||||
"""{"group": "connection", "order": 4, "airbyte_secret": true, "always_show": true}""",
|
||||
)
|
||||
val credentialsJson: String? = null
|
||||
|
||||
@get:JsonSchemaTitle("Transformation Query Run Type")
|
||||
@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.""",
|
||||
)
|
||||
@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)""",
|
||||
)
|
||||
@get:JsonProperty("raw_data_dataset")
|
||||
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 7}""")
|
||||
val rawTableDataset: String? = null
|
||||
|
||||
@get:JsonSchemaTitle(
|
||||
"Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)",
|
||||
)
|
||||
@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""",
|
||||
)
|
||||
@get:JsonProperty("disable_type_dedupe")
|
||||
@get:JsonSchemaInject(json = """{"group": "advanced", "order": 8, "default": false}""")
|
||||
val disableTypingDeduping: Boolean? = null
|
||||
}
|
||||
|
||||
@JsonTypeInfo(
|
||||
use = JsonTypeInfo.Id.NAME,
|
||||
include = JsonTypeInfo.As.EXISTING_PROPERTY,
|
||||
property = "method",
|
||||
)
|
||||
@JsonSubTypes(
|
||||
JsonSubTypes.Type(value = BatchedStandardInsertSpecification::class, name = "Standard"),
|
||||
JsonSubTypes.Type(value = GcsStagingSpecification::class, name = "GCS Staging"),
|
||||
)
|
||||
sealed class LoadingMethodSpecification(@JsonProperty("method") val method: LoadingMethod) {
|
||||
enum class LoadingMethod(@get:JsonValue val typeName: String) {
|
||||
BATCHED_STANDARD_INSERT("Standard"),
|
||||
GCS("GCS Staging"),
|
||||
}
|
||||
}
|
||||
|
||||
@JsonSchemaTitle("Batched Standard Inserts")
|
||||
@JsonSchemaDescription(
|
||||
"Direct loading using batched SQL INSERT statements. This method uses the BigQuery driver to convert large INSERT statements into file uploads automatically.",
|
||||
)
|
||||
class BatchedStandardInsertSpecification :
|
||||
LoadingMethodSpecification(LoadingMethod.BATCHED_STANDARD_INSERT)
|
||||
|
||||
@JsonSchemaTitle("GCS Staging")
|
||||
@JsonSchemaDescription(
|
||||
"Writes large batches of records to a file, uploads the file to GCS, then uses COPY INTO to load your data into BigQuery.",
|
||||
)
|
||||
class GcsStagingSpecification :
|
||||
GcsCommonSpecification, LoadingMethodSpecification(LoadingMethod.GCS) {
|
||||
@get:JsonSchemaTitle("GCS Tmp Files Post-Processing")
|
||||
@get:JsonPropertyDescription(
|
||||
"""This upload method is supposed to temporary store records in GCS bucket. By this select you can chose if these records should be removed from GCS when migration has finished. The default "Delete all tmp files from GCS" value is used if not set explicitly.""",
|
||||
)
|
||||
// yes, this is mixed underscore+hyphen.
|
||||
@get:JsonProperty("keep_files_in_gcs-bucket", defaultValue = "Delete all tmp files from GCS")
|
||||
@get:JsonSchemaInject(json = """{"order": 3}""")
|
||||
val filePostProcessing: GcsFilePostProcessing? = null
|
||||
override val gcsBucketName: String = ""
|
||||
override val path: String = ""
|
||||
override val credential: GcsAuthSpecification =
|
||||
GcsHmacKeySpecification(accessKeyId = "", secretAccessKey = "")
|
||||
}
|
||||
|
||||
// bigquery supports a subset of GCS regions.
|
||||
// See https://cloud.google.com/bigquery/docs/locations#supported_locations
|
||||
enum class BigqueryRegion(@get:JsonValue val region: String, val gcsRegion: GcsRegion) {
|
||||
EU("EU", GcsRegion.EU),
|
||||
US("US", GcsRegion.US),
|
||||
AFRICA_SOUTH1("africa-south1", GcsRegion.AFRICA_SOUTH1),
|
||||
ASIA_EAST1("asia-east1", GcsRegion.ASIA_EAST1),
|
||||
ASIA_EAST2("asia-east2", GcsRegion.ASIA_EAST2),
|
||||
ASIA_NORTHEAST1("asia-northeast1", GcsRegion.ASIA_NORTHEAST1),
|
||||
ASIA_NORTHEAST2("asia-northeast2", GcsRegion.ASIA_NORTHEAST2),
|
||||
ASIA_NORTHEAST3("asia-northeast3", GcsRegion.ASIA_NORTHEAST3),
|
||||
ASIA_SOUTH1("asia-south1", GcsRegion.ASIA_SOUTH1),
|
||||
ASIA_SOUTH2("asia-south2", GcsRegion.ASIA_SOUTH2),
|
||||
ASIA_SOUTHEAST1("asia-southeast1", GcsRegion.ASIA_SOUTHEAST1),
|
||||
ASIA_SOUTHEAST2("asia-southeast2", GcsRegion.ASIA_SOUTHEAST2),
|
||||
AUSTRALIA_SOUTHEAST1("australia-southeast1", GcsRegion.AUSTRALIA_SOUTHEAST1),
|
||||
AUSTRALIA_SOUTHEAST2("australia-southeast2", GcsRegion.AUSTRALIA_SOUTHEAST2),
|
||||
EUROPE_CENTRAL2("europe-central2", GcsRegion.EUROPE_CENTRAL2),
|
||||
EUROPE_NORTH1("europe-north1", GcsRegion.EUROPE_NORTH1),
|
||||
EUROPE_NORTH2("europe-north2", GcsRegion.EUROPE_NORTH2),
|
||||
EUROPE_SOUTHWEST1("europe-southwest1", GcsRegion.EUROPE_SOUTHWEST1),
|
||||
EUROPE_WEST1("europe-west1", GcsRegion.EUROPE_WEST1),
|
||||
EUROPE_WEST2("europe-west2", GcsRegion.EUROPE_WEST2),
|
||||
EUROPE_WEST3("europe-west3", GcsRegion.EUROPE_WEST3),
|
||||
EUROPE_WEST4("europe-west4", GcsRegion.EUROPE_WEST4),
|
||||
EUROPE_WEST6("europe-west6", GcsRegion.EUROPE_WEST6),
|
||||
EUROPE_WEST8("europe-west8", GcsRegion.EUROPE_WEST8),
|
||||
EUROPE_WEST9("europe-west9", GcsRegion.EUROPE_WEST9),
|
||||
EUROPE_WEST10("europe-west10", GcsRegion.EUROPE_WEST10),
|
||||
EUROPE_WEST12("europe-west12", GcsRegion.EUROPE_WEST12),
|
||||
ME_CENTRAL1("me-central1", GcsRegion.ME_CENTRAL1),
|
||||
ME_CENTRAL2("me-central2", GcsRegion.ME_CENTRAL2),
|
||||
ME_WEST1("me-west1", GcsRegion.ME_WEST1),
|
||||
NORTHAMERICA_NORTHEAST1("northamerica-northeast1", GcsRegion.NORTHAMERICA_NORTHEAST1),
|
||||
NORTHAMERICA_NORTHEAST2("northamerica-northeast2", GcsRegion.NORTHAMERICA_NORTHEAST2),
|
||||
NORTHAMERICA_SOUTH1("northamerica-south1", GcsRegion.NORTHAMERICA_SOUTH1),
|
||||
SOUTHAMERICA_EAST1("southamerica-east1", GcsRegion.SOUTHAMERICA_EAST1),
|
||||
SOUTHAMERICA_WEST1("southamerica-west1", GcsRegion.SOUTHAMERICA_WEST1),
|
||||
US_CENTRAL1("us-central1", GcsRegion.US_CENTRAL1),
|
||||
US_EAST1("us-east1", GcsRegion.US_EAST1),
|
||||
US_EAST4("us-east4", GcsRegion.US_EAST4),
|
||||
US_EAST5("us-east5", GcsRegion.US_EAST5),
|
||||
US_SOUTH1("us-south1", GcsRegion.US_SOUTH1),
|
||||
US_WEST1("us-west1", GcsRegion.US_WEST1),
|
||||
US_WEST2("us-west2", GcsRegion.US_WEST2),
|
||||
US_WEST3("us-west3", GcsRegion.US_WEST3),
|
||||
US_WEST4("us-west4", GcsRegion.US_WEST4),
|
||||
}
|
||||
|
||||
enum class GcsFilePostProcessing(@get:JsonValue val postProcesing: String) {
|
||||
DELETE("Delete all tmp files from GCS"),
|
||||
KEEP("Keep all tmp files in GCS"),
|
||||
}
|
||||
|
||||
enum class TransformationPriority(@get:JsonValue val transformationPriority: String) {
|
||||
INTERACTIVE("interactive"),
|
||||
BATCH("batch")
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigquerySpecificationExtension : DestinationSpecificationExtension {
|
||||
override val supportedSyncModes =
|
||||
listOf(
|
||||
DestinationSyncMode.OVERWRITE,
|
||||
DestinationSyncMode.APPEND,
|
||||
DestinationSyncMode.APPEND_DEDUP,
|
||||
)
|
||||
override val supportsIncremental = true
|
||||
override val groups =
|
||||
listOf(
|
||||
DestinationSpecificationExtension.Group("connection", "Connection"),
|
||||
DestinationSpecificationExtension.Group("advanced", "Advanced"),
|
||||
)
|
||||
}
|
||||
|
||||
/**
|
||||
* A custom JSON deserializer, which can write any JSON value into a String field. In particular, it
|
||||
* passes String values through unchanged, but serializes all other values.
|
||||
*
|
||||
* If you don't do this, then Jackson will choke on object values.
|
||||
*/
|
||||
class RawJsonDeserializer : JsonDeserializer<String?>() {
|
||||
@Throws(IOException::class, JsonProcessingException::class)
|
||||
override fun deserialize(jp: JsonParser, ctxt: DeserializationContext?): String {
|
||||
val node: JsonNode = Jsons.readTree(jp)
|
||||
if (node.isTextual) {
|
||||
return node.asText()
|
||||
}
|
||||
return Jsons.writeValueAsString(node)
|
||||
}
|
||||
}
|
||||
@@ -1,123 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.Job
|
||||
import com.google.cloud.bigquery.JobConfiguration
|
||||
import com.google.cloud.bigquery.JobId
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.JobStatistics
|
||||
import com.google.cloud.bigquery.JobStatus
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.orchestration.db.DatabaseHandler
|
||||
import io.airbyte.cdk.load.orchestration.db.Sql
|
||||
import io.airbyte.cdk.util.ConnectorExceptionUtil
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.util.UUID
|
||||
import kotlin.math.min
|
||||
import kotlinx.coroutines.coroutineScope
|
||||
import kotlinx.coroutines.launch
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
class BigQueryDatabaseHandler(private val bq: BigQuery, private val datasetLocation: String) :
|
||||
DatabaseHandler {
|
||||
@Throws(InterruptedException::class)
|
||||
override fun execute(sql: Sql) {
|
||||
val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION")
|
||||
if (transactions.isEmpty()) {
|
||||
return
|
||||
}
|
||||
val queryId = UUID.randomUUID()
|
||||
val statement = java.lang.String.join("\n", transactions)
|
||||
logger.debug { "Executing sql $queryId: $statement" }
|
||||
|
||||
/*
|
||||
* If you run a query like CREATE SCHEMA ... OPTIONS(location=foo); CREATE TABLE ...;, bigquery
|
||||
* doesn't do a good job of inferring the query location. Pass it in explicitly.
|
||||
*/
|
||||
var job =
|
||||
bq.create(
|
||||
JobInfo.of(
|
||||
JobId.newBuilder().setLocation(datasetLocation).build(),
|
||||
QueryJobConfiguration.of(statement)
|
||||
)
|
||||
)
|
||||
// job.waitFor() gets stuck forever in some failure cases, so manually poll the job instead.
|
||||
while (JobStatus.State.DONE != job.status.state) {
|
||||
Thread.sleep(1000L)
|
||||
job = job.reload()
|
||||
}
|
||||
job.status.error?.let {
|
||||
throw BigQueryException(listOf(job.status.error) + job.status.executionErrors)
|
||||
}
|
||||
|
||||
val statistics = job.getStatistics<JobStatistics.QueryStatistics>()
|
||||
logger.debug {
|
||||
"Root-level job $queryId completed in ${statistics.endTime - statistics.startTime} ms; processed ${statistics.totalBytesProcessed} bytes; billed for ${statistics.totalBytesBilled} bytes"
|
||||
}
|
||||
|
||||
// SQL transactions can spawn child jobs, which are billed individually. Log their stats
|
||||
// too.
|
||||
if (statistics.numChildJobs != null) {
|
||||
// There isn't (afaict) anything resembling job.getChildJobs(), so we have to ask bq for
|
||||
// them
|
||||
bq.listJobs(BigQuery.JobListOption.parentJobId(job.jobId.job))
|
||||
.iterateAll()
|
||||
.sortedBy { it.getStatistics<JobStatistics>().endTime }
|
||||
.forEach { childJob: Job ->
|
||||
val configuration = childJob.getConfiguration<JobConfiguration>()
|
||||
if (configuration is QueryJobConfiguration) {
|
||||
val childQueryStats =
|
||||
childJob.getStatistics<JobStatistics.QueryStatistics>()
|
||||
var truncatedQuery: String =
|
||||
configuration.query
|
||||
.replace("\\s+".toRegex(), " ")
|
||||
.substring(
|
||||
0,
|
||||
min(100.0, configuration.query.length.toDouble()).toInt()
|
||||
)
|
||||
if (truncatedQuery != configuration.query) {
|
||||
truncatedQuery += "..."
|
||||
}
|
||||
logger.debug {
|
||||
"Child sql $truncatedQuery completed in ${childQueryStats.endTime - childQueryStats.startTime} ms; processed ${childQueryStats.totalBytesProcessed} bytes; billed for ${childQueryStats.totalBytesBilled} bytes"
|
||||
}
|
||||
} else {
|
||||
// other job types are extract/copy/load
|
||||
// we're probably not using them, but handle just in case?
|
||||
val childJobStats = childJob.getStatistics<JobStatistics>()
|
||||
logger.debug {
|
||||
"Non-query child job (${configuration.type}) completed in ${childJobStats.endTime - childJobStats.startTime} ms"
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
override suspend fun createNamespaces(namespaces: Collection<String>) {
|
||||
coroutineScope {
|
||||
namespaces.forEach { dataset ->
|
||||
launch {
|
||||
logger.info { "Creating dataset if not present $dataset" }
|
||||
try {
|
||||
BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation)
|
||||
} catch (e: BigQueryException) {
|
||||
if (
|
||||
ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code)
|
||||
) {
|
||||
throw ConfigErrorException(e.message!!, e)
|
||||
} else {
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,427 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.google.cloud.bigquery.*
|
||||
import com.google.common.annotations.VisibleForTesting
|
||||
import com.google.common.collect.Streams
|
||||
import io.airbyte.cdk.integrations.base.AirbyteExceptionHandler
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.util.ConnectorExceptionUtil
|
||||
import io.airbyte.commons.exceptions.ConfigErrorException
|
||||
import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation
|
||||
import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation.Companion.TMP_TABLE_SUFFIX
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.*
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsIgnoreCase
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.matchingKey
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState
|
||||
import java.math.BigInteger
|
||||
import java.util.*
|
||||
import java.util.function.Consumer
|
||||
import java.util.function.Function
|
||||
import java.util.function.Predicate
|
||||
import java.util.stream.Collectors
|
||||
import java.util.stream.Stream
|
||||
import kotlin.math.min
|
||||
import org.apache.commons.text.StringSubstitutor
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
class BigQueryDestinationHandler(private val bq: BigQuery, private val datasetLocation: String) :
|
||||
DestinationHandler<BigQueryDestinationState> {
|
||||
fun findExistingTable(id: StreamId): Optional<TableDefinition> {
|
||||
val table = bq.getTable(id.finalNamespace, id.finalName)
|
||||
return Optional.ofNullable(table).map { obj: Table -> obj.getDefinition() }
|
||||
}
|
||||
|
||||
fun isFinalTableEmpty(id: StreamId): Boolean {
|
||||
return BigInteger.ZERO == bq.getTable(TableId.of(id.finalNamespace, id.finalName)).numRows
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
fun getInitialRawTableState(id: StreamId, suffix: String): InitialRawTableStatus {
|
||||
bq.getTable(TableId.of(id.rawNamespace, id.rawName + suffix))
|
||||
?: // Table doesn't exist. There are no unprocessed records, and no timestamp.
|
||||
return InitialRawTableStatus(false, false, Optional.empty())
|
||||
|
||||
val unloadedRecordTimestamp =
|
||||
bq.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
java.util.Map.of<String, String>(
|
||||
"raw_table",
|
||||
id.rawTableId(BigQuerySqlGenerator.Companion.QUOTE, suffix)
|
||||
)
|
||||
)
|
||||
.replace( // bigquery timestamps have microsecond precision
|
||||
"""
|
||||
SELECT TIMESTAMP_SUB(MIN(_airbyte_extracted_at), INTERVAL 1 MICROSECOND)
|
||||
FROM ${'$'}{raw_table}
|
||||
WHERE _airbyte_loaded_at IS NULL
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.iterateAll()
|
||||
.iterator()
|
||||
.next()
|
||||
.first()
|
||||
// If this value is null, then there are no records with null loaded_at.
|
||||
// If it's not null, then we can return immediately - we've found some unprocessed records
|
||||
// and their
|
||||
// timestamp.
|
||||
if (!unloadedRecordTimestamp.isNull) {
|
||||
return InitialRawTableStatus(
|
||||
true,
|
||||
true,
|
||||
Optional.of(unloadedRecordTimestamp.timestampInstant)
|
||||
)
|
||||
}
|
||||
|
||||
val loadedRecordTimestamp =
|
||||
bq.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
java.util.Map.of<String, String>(
|
||||
"raw_table",
|
||||
id.rawTableId(BigQuerySqlGenerator.Companion.QUOTE, suffix)
|
||||
)
|
||||
)
|
||||
.replace(
|
||||
"""
|
||||
SELECT MAX(_airbyte_extracted_at)
|
||||
FROM ${'$'}{raw_table}
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
.iterateAll()
|
||||
.iterator()
|
||||
.next()
|
||||
.first()
|
||||
// We know (from the previous query) that all records have been processed by T+D already.
|
||||
// So we just need to get the timestamp of the most recent record.
|
||||
return if (loadedRecordTimestamp.isNull) {
|
||||
// Null timestamp because the table is empty. T+D can process the entire raw table
|
||||
// during this sync.
|
||||
InitialRawTableStatus(true, false, Optional.empty())
|
||||
} else {
|
||||
// The raw table already has some records. T+D can skip all records with timestamp <=
|
||||
// this value.
|
||||
InitialRawTableStatus(true, false, Optional.of(loadedRecordTimestamp.timestampInstant))
|
||||
}
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
override fun execute(sql: Sql) {
|
||||
val transactions = sql.asSqlStrings("BEGIN TRANSACTION", "COMMIT TRANSACTION")
|
||||
if (transactions.isEmpty()) {
|
||||
return
|
||||
}
|
||||
val queryId = UUID.randomUUID()
|
||||
val statement = java.lang.String.join("\n", transactions)
|
||||
LOGGER.debug("Executing sql {}: {}", queryId, statement)
|
||||
|
||||
/*
|
||||
* If you run a query like CREATE SCHEMA ... OPTIONS(location=foo); CREATE TABLE ...;, bigquery
|
||||
* doesn't do a good job of inferring the query location. Pass it in explicitly.
|
||||
*/
|
||||
var job =
|
||||
bq.create(
|
||||
JobInfo.of(
|
||||
JobId.newBuilder().setLocation(datasetLocation).build(),
|
||||
QueryJobConfiguration.newBuilder(statement).build()
|
||||
)
|
||||
)
|
||||
AirbyteExceptionHandler.addStringForDeinterpolation(job.etag)
|
||||
// job.waitFor() gets stuck forever in some failure cases, so manually poll the job instead.
|
||||
while (JobStatus.State.DONE != job.status.state) {
|
||||
Thread.sleep(1000L)
|
||||
job = job.reload()
|
||||
}
|
||||
if (job.status.error != null) {
|
||||
throw BigQueryException(
|
||||
Streams.concat(Stream.of(job.status.error), job.status.executionErrors.stream())
|
||||
.toList()
|
||||
)
|
||||
}
|
||||
|
||||
val statistics = job.getStatistics<JobStatistics.QueryStatistics>()
|
||||
LOGGER.debug(
|
||||
"Root-level job {} completed in {} ms; processed {} bytes; billed for {} bytes",
|
||||
queryId,
|
||||
statistics.endTime - statistics.startTime,
|
||||
statistics.totalBytesProcessed,
|
||||
statistics.totalBytesBilled
|
||||
)
|
||||
|
||||
// SQL transactions can spawn child jobs, which are billed individually. Log their stats
|
||||
// too.
|
||||
if (statistics.numChildJobs != null) {
|
||||
// There isn't (afaict) anything resembling job.getChildJobs(), so we have to ask bq for
|
||||
// them
|
||||
bq.listJobs(BigQuery.JobListOption.parentJobId(job.jobId.job))
|
||||
.streamAll()
|
||||
.sorted(
|
||||
Comparator.comparing { childJob: Job ->
|
||||
childJob.getStatistics<JobStatistics>().endTime
|
||||
}
|
||||
)
|
||||
.forEach { childJob: Job ->
|
||||
val configuration = childJob.getConfiguration<JobConfiguration>()
|
||||
if (configuration is QueryJobConfiguration) {
|
||||
val childQueryStats =
|
||||
childJob.getStatistics<JobStatistics.QueryStatistics>()
|
||||
var truncatedQuery: String =
|
||||
configuration.query
|
||||
.replace("\n".toRegex(), " ")
|
||||
.replace(" +".toRegex(), " ")
|
||||
.substring(
|
||||
0,
|
||||
min(100.0, configuration.query.length.toDouble()).toInt()
|
||||
)
|
||||
if (truncatedQuery != configuration.query) {
|
||||
truncatedQuery += "..."
|
||||
}
|
||||
LOGGER.debug(
|
||||
"Child sql {} completed in {} ms; processed {} bytes; billed for {} bytes",
|
||||
truncatedQuery,
|
||||
childQueryStats.endTime - childQueryStats.startTime,
|
||||
childQueryStats.totalBytesProcessed,
|
||||
childQueryStats.totalBytesBilled
|
||||
)
|
||||
} else {
|
||||
// other job types are extract/copy/load
|
||||
// we're probably not using them, but handle just in case?
|
||||
val childJobStats = childJob.getStatistics<JobStatistics>()
|
||||
LOGGER.debug(
|
||||
"Non-query child job ({}) completed in {} ms",
|
||||
configuration.type,
|
||||
childJobStats.endTime - childJobStats.startTime
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun gatherInitialState(
|
||||
streamConfigs: List<StreamConfig>
|
||||
): List<DestinationInitialStatus<BigQueryDestinationState>> {
|
||||
val initialStates: MutableList<DestinationInitialStatus<BigQueryDestinationState>> =
|
||||
ArrayList()
|
||||
for (streamConfig in streamConfigs) {
|
||||
val id = streamConfig.id
|
||||
val finalTable = findExistingTable(id)
|
||||
val rawTableState = getInitialRawTableState(id, "")
|
||||
val tempRawTableState =
|
||||
getInitialRawTableState(id, AbstractStreamOperation.TMP_TABLE_SUFFIX)
|
||||
initialStates.add(
|
||||
DestinationInitialStatus(
|
||||
streamConfig,
|
||||
finalTable.isPresent,
|
||||
rawTableState,
|
||||
tempRawTableState,
|
||||
finalTable.isPresent &&
|
||||
!existingSchemaMatchesStreamConfig(streamConfig, finalTable.get()),
|
||||
finalTable.isEmpty ||
|
||||
isFinalTableEmpty(
|
||||
id
|
||||
), // Return a default state blob since we don't actually track state.
|
||||
BigQueryDestinationState(false),
|
||||
// for now, just use 0. this means we will always use a temp final table.
|
||||
// platform has a workaround for this, so it's OK.
|
||||
// TODO only fetch this on truncate syncs
|
||||
// TODO once we have destination state, use that instead of a query
|
||||
finalTableGenerationId = 0,
|
||||
// temp table is always empty until we commit, so always return null
|
||||
finalTempTableGenerationId = null,
|
||||
)
|
||||
)
|
||||
}
|
||||
return initialStates
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun commitDestinationStates(
|
||||
destinationStates: Map<StreamId, BigQueryDestinationState>
|
||||
) {
|
||||
// Intentionally do nothing. Bigquery doesn't actually support destination states.
|
||||
}
|
||||
|
||||
@Throws(TableNotMigratedException::class)
|
||||
private fun existingSchemaMatchesStreamConfig(
|
||||
stream: StreamConfig,
|
||||
existingTable: TableDefinition
|
||||
): Boolean {
|
||||
val alterTableReport = buildAlterTableReport(stream, existingTable)
|
||||
var tableClusteringMatches = false
|
||||
var tablePartitioningMatches = false
|
||||
if (existingTable is StandardTableDefinition) {
|
||||
tableClusteringMatches = clusteringMatches(stream, existingTable)
|
||||
tablePartitioningMatches = partitioningMatches(existingTable)
|
||||
}
|
||||
LOGGER.info(
|
||||
"Alter Table Report {} {} {}; Clustering {}; Partitioning {}",
|
||||
alterTableReport.columnsToAdd,
|
||||
alterTableReport.columnsToRemove,
|
||||
alterTableReport.columnsToChangeType,
|
||||
tableClusteringMatches,
|
||||
tablePartitioningMatches
|
||||
)
|
||||
|
||||
return alterTableReport.isNoOp && tableClusteringMatches && tablePartitioningMatches
|
||||
}
|
||||
|
||||
fun buildAlterTableReport(
|
||||
stream: StreamConfig,
|
||||
existingTable: TableDefinition
|
||||
): AlterTableReport {
|
||||
val pks = getPks(stream)
|
||||
|
||||
val streamSchema: Map<String, StandardSQLTypeName> =
|
||||
stream.columns.entries.associate {
|
||||
it.key.name to BigQuerySqlGenerator.toDialectType(it.value)
|
||||
}
|
||||
|
||||
val existingSchema =
|
||||
existingTable.schema!!
|
||||
.fields
|
||||
.stream()
|
||||
.collect(
|
||||
Collectors.toMap(
|
||||
Function { field: Field -> field.name },
|
||||
Function { field: Field -> field.type.standardType }
|
||||
)
|
||||
)
|
||||
|
||||
// Columns in the StreamConfig that don't exist in the TableDefinition
|
||||
val columnsToAdd =
|
||||
streamSchema.keys
|
||||
.stream()
|
||||
.filter { name: String -> !containsIgnoreCase(existingSchema.keys, name) }
|
||||
.collect(Collectors.toSet())
|
||||
|
||||
// Columns in the current schema that are no longer in the StreamConfig
|
||||
val columnsToRemove =
|
||||
existingSchema.keys
|
||||
.stream()
|
||||
.filter { name: String ->
|
||||
!containsIgnoreCase(streamSchema.keys, name) &&
|
||||
!containsIgnoreCase(JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS, name)
|
||||
}
|
||||
.collect(Collectors.toSet<String>())
|
||||
|
||||
// Columns that are typed differently than the StreamConfig
|
||||
val columnsToChangeType =
|
||||
Stream.concat(
|
||||
streamSchema.keys
|
||||
.stream() // If it's not in the existing schema, it should already be in the
|
||||
// columnsToAdd Set
|
||||
.filter { name: String ->
|
||||
matchingKey(
|
||||
existingSchema.keys,
|
||||
name
|
||||
) // if it does exist, only include it in this set if the type (the
|
||||
// value in each respective map)
|
||||
// is different between the stream and existing schemas
|
||||
.map { key: String ->
|
||||
existingSchema[key] != streamSchema[name]
|
||||
} // if there is no matching key, then don't include it because it
|
||||
// is probably already in columnsToAdd
|
||||
.orElse(false)
|
||||
}, // OR columns that used to have a non-null constraint and shouldn't
|
||||
// (https://github.com/airbytehq/airbyte/pull/31082)
|
||||
|
||||
existingTable.schema!!
|
||||
.fields
|
||||
.stream()
|
||||
.filter { field: Field -> pks.contains(field.name) }
|
||||
.filter { field: Field -> field.mode == Field.Mode.REQUIRED }
|
||||
.map { obj: Field -> obj.name }
|
||||
)
|
||||
.collect(Collectors.toSet())
|
||||
|
||||
val isDestinationV2Format = schemaContainAllFinalTableV2AirbyteColumns(existingSchema.keys)
|
||||
|
||||
return AlterTableReport(
|
||||
columnsToAdd,
|
||||
columnsToRemove,
|
||||
columnsToChangeType,
|
||||
isDestinationV2Format
|
||||
)
|
||||
}
|
||||
|
||||
override fun createNamespaces(schemas: Set<String>) {
|
||||
schemas.forEach(Consumer { dataset: String -> this.createDataset(dataset) })
|
||||
}
|
||||
|
||||
private fun createDataset(dataset: String) {
|
||||
LOGGER.info("Creating dataset if not present {}", dataset)
|
||||
try {
|
||||
BigQueryUtils.getOrCreateDataset(bq, dataset, datasetLocation)
|
||||
} catch (e: BigQueryException) {
|
||||
if (ConnectorExceptionUtil.HTTP_AUTHENTICATION_ERROR_CODES.contains(e.code)) {
|
||||
throw ConfigErrorException(e.message!!, e)
|
||||
} else {
|
||||
throw e
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDestinationHandler::class.java)
|
||||
|
||||
@JvmStatic
|
||||
@VisibleForTesting
|
||||
fun clusteringMatches(
|
||||
stream: StreamConfig,
|
||||
existingTable: StandardTableDefinition
|
||||
): Boolean {
|
||||
return (existingTable.clustering != null &&
|
||||
containsAllIgnoreCase(
|
||||
HashSet<String>(existingTable.clustering!!.fields),
|
||||
BigQuerySqlGenerator.Companion.clusteringColumns(stream)
|
||||
))
|
||||
}
|
||||
|
||||
@JvmStatic
|
||||
@VisibleForTesting
|
||||
fun partitioningMatches(existingTable: StandardTableDefinition): Boolean {
|
||||
return existingTable.timePartitioning != null &&
|
||||
existingTable.timePartitioning!!
|
||||
.field
|
||||
.equals("_airbyte_extracted_at", ignoreCase = true) &&
|
||||
TimePartitioning.Type.DAY == existingTable.timePartitioning!!.type
|
||||
}
|
||||
|
||||
/**
|
||||
* Checks the schema to determine whether the table contains all expected final table
|
||||
* airbyte columns
|
||||
*
|
||||
* @param columnNames the column names of the schema to check
|
||||
* @return whether all the [JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS] are present
|
||||
*/
|
||||
@VisibleForTesting
|
||||
@JvmStatic
|
||||
fun schemaContainAllFinalTableV2AirbyteColumns(columnNames: Collection<String>?): Boolean {
|
||||
return JavaBaseConstants.V2_FINAL_TABLE_METADATA_COLUMNS.stream()
|
||||
.allMatch(
|
||||
Predicate<String> { column: String? ->
|
||||
containsIgnoreCase(columnNames!!, column!!)
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
private fun getPks(stream: StreamConfig): Set<String> {
|
||||
return stream.primaryKey.map(ColumnId::name).toSet()
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -5,57 +5,67 @@ 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.base.destination.typing_deduping.*
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.Array
|
||||
import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer
|
||||
import java.time.Instant
|
||||
import java.util.*
|
||||
import java.util.stream.Collectors
|
||||
import java.util.stream.Stream
|
||||
import org.apache.commons.lang3.StringUtils
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
class BigQuerySqlGenerator
|
||||
/**
|
||||
* @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.
|
||||
* @param datasetLocation This is technically redundant with [BigQueryDestinationHandler] setting
|
||||
* the query execution location, but let's be explicit since this is typically a compliance
|
||||
* requirement.
|
||||
*/
|
||||
class BigQuerySqlGenerator(private val projectId: String?, private val datasetLocation: String?) :
|
||||
TypingDedupingSqlGenerator {
|
||||
(private val projectId: String?, private val datasetLocation: String?) : SqlGenerator {
|
||||
private val CDC_DELETED_AT_COLUMN = buildColumnId("_ab_cdc_deleted_at")
|
||||
|
||||
private val LOGGER: Logger = LoggerFactory.getLogger(BigQuerySqlGenerator::class.java)
|
||||
|
||||
override fun buildStreamId(
|
||||
namespace: String,
|
||||
name: String,
|
||||
rawNamespaceOverride: String
|
||||
): StreamId {
|
||||
return StreamId(
|
||||
nameTransformer.getNamespace(namespace),
|
||||
nameTransformer.convertStreamName(name),
|
||||
nameTransformer.getNamespace(rawNamespaceOverride),
|
||||
nameTransformer.convertStreamName(StreamId.concatenateRawTableName(namespace, name)),
|
||||
namespace,
|
||||
name
|
||||
)
|
||||
}
|
||||
|
||||
override fun buildColumnId(name: String, suffix: String?): ColumnId {
|
||||
val nameWithSuffix = name + suffix
|
||||
return ColumnId(
|
||||
nameTransformer.getIdentifier(nameWithSuffix),
|
||||
name, // Bigquery columns are case-insensitive, so do all our validation on the
|
||||
// lowercased name
|
||||
nameTransformer.getIdentifier(nameWithSuffix.lowercase(Locale.getDefault()))
|
||||
)
|
||||
}
|
||||
|
||||
private fun extractAndCast(
|
||||
columnName: String,
|
||||
column: ColumnId,
|
||||
airbyteType: AirbyteType,
|
||||
forceSafeCast: Boolean
|
||||
): String {
|
||||
if (airbyteType is UnionType && airbyteType.isLegacyUnion) {
|
||||
if (airbyteType is Union) {
|
||||
// This is guaranteed to not be a Union, so we won't recurse infinitely
|
||||
val chosenType: AirbyteType = airbyteType.chooseType()
|
||||
return extractAndCast(columnName, chosenType, forceSafeCast)
|
||||
return extractAndCast(column, chosenType, forceSafeCast)
|
||||
}
|
||||
val jsonPathEscapedColumnName = escapeColumnNameForJsonPath(columnName)
|
||||
val columnName = escapeColumnNameForJsonPath(column.originalName)
|
||||
|
||||
if (airbyteType.isObject) {
|
||||
if (airbyteType is Struct) {
|
||||
// We need to validate that the struct is actually a struct.
|
||||
// Note that struct columns are actually nullable in two ways. For a column `foo`:
|
||||
// {foo: null} and {} are both valid, and are both written to the final table as a SQL
|
||||
@@ -65,52 +75,53 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
// JSON_QUERY(JSON'{"foo": null}', '$."foo"') returns a JSON null.
|
||||
return """
|
||||
PARSE_JSON(CASE
|
||||
WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"') IS NULL
|
||||
OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"'), wide_number_mode=>'round')) != 'object'
|
||||
WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL
|
||||
OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'object'
|
||||
THEN NULL
|
||||
ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"')
|
||||
ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"')
|
||||
END, wide_number_mode=>'round')
|
||||
""".trimIndent()
|
||||
}
|
||||
|
||||
if (airbyteType.isArray) {
|
||||
if (airbyteType is Array) {
|
||||
// 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'
|
||||
WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL
|
||||
OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'array'
|
||||
THEN NULL
|
||||
ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"')
|
||||
ELSE JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"')
|
||||
END, wide_number_mode=>'round')
|
||||
""".trimIndent()
|
||||
}
|
||||
|
||||
if (airbyteType is UnionType || airbyteType is UnknownType) {
|
||||
if (airbyteType is UnsupportedOneOf || airbyteType === AirbyteProtocolType.UNKNOWN) {
|
||||
// JSON_QUERY returns a SQL null if the field contains a JSON null, so we actually parse
|
||||
// the
|
||||
// airbyte_data to json
|
||||
// and json_query it directly (which preserves nulls correctly).
|
||||
return """JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$jsonPathEscapedColumnName"')"""
|
||||
return """JSON_QUERY(PARSE_JSON(`_airbyte_data`, wide_number_mode=>'round'), '${'$'}."$columnName"')"""
|
||||
}
|
||||
|
||||
if (airbyteType is StringType) {
|
||||
if (airbyteType === AirbyteProtocolType.STRING) {
|
||||
// Special case String to only use json value for type string and parse the json for
|
||||
// others
|
||||
// Naive json_value returns NULL for object/array values and json_query adds escaped
|
||||
// quotes to the string.
|
||||
// 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"')
|
||||
WHEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"') IS NULL
|
||||
OR JSON_TYPE(PARSE_JSON(JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"'), wide_number_mode=>'round')) != 'string'
|
||||
THEN JSON_QUERY(`_airbyte_data`, '${'$'}."$columnName"')
|
||||
ELSE
|
||||
JSON_VALUE(`_airbyte_data`, '${'$'}."$jsonPathEscapedColumnName"')
|
||||
JSON_VALUE(`_airbyte_data`, '${'$'}."$columnName"')
|
||||
END)
|
||||
""".trimIndent()
|
||||
}
|
||||
|
||||
val dialectType = toDialectType(airbyteType)
|
||||
val baseTyping = """JSON_VALUE(`_airbyte_data`, '$."$jsonPathEscapedColumnName"')"""
|
||||
val baseTyping = """JSON_VALUE(`_airbyte_data`, '$."$columnName"')"""
|
||||
return if (dialectType == StandardSQLTypeName.STRING) {
|
||||
// json_value implicitly returns a string, so we don't need to cast it.
|
||||
baseTyping
|
||||
@@ -120,163 +131,99 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
}
|
||||
|
||||
override fun createFinalTable(
|
||||
stream: DestinationStream,
|
||||
tableName: TableName,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
finalTableSuffix: String,
|
||||
replace: Boolean
|
||||
): Sql {
|
||||
val columnDeclarations = columnsAndTypes(stream, columnNameMapping)
|
||||
override fun createTable(stream: StreamConfig, suffix: String, force: Boolean): Sql {
|
||||
val columnDeclarations = columnsAndTypes(stream)
|
||||
val clusterConfig =
|
||||
clusteringColumns(stream, columnNameMapping)
|
||||
clusteringColumns(stream)
|
||||
.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(),
|
||||
val forceCreateTable = if (force) "OR REPLACE" else ""
|
||||
val finalTableId = stream.id.finalTableId(QUOTE, suffix)
|
||||
return Sql.of(
|
||||
"""
|
||||
CREATE $forceCreateTable TABLE `$projectId`.$finalTableId (
|
||||
_airbyte_raw_id STRING NOT NULL,
|
||||
_airbyte_extracted_at TIMESTAMP NOT NULL,
|
||||
_airbyte_meta JSON NOT NULL,
|
||||
_airbyte_generation_id INTEGER,
|
||||
$columnDeclarations
|
||||
)
|
||||
PARTITION BY (DATE_TRUNC(_airbyte_extracted_at, DAY))
|
||||
CLUSTER BY $clusterConfig;
|
||||
""".trimIndent()
|
||||
)
|
||||
}
|
||||
|
||||
private fun columnsAndTypes(
|
||||
stream: DestinationStream,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
): String {
|
||||
return stream.schema
|
||||
.asColumns()
|
||||
.map { (fieldName, type) ->
|
||||
val columnName = columnNameMapping[fieldName]!!
|
||||
val typeName = toDialectType(type.type).name
|
||||
"`$columnName` $typeName"
|
||||
private fun columnsAndTypes(stream: StreamConfig): String {
|
||||
return stream.columns.entries
|
||||
.stream()
|
||||
.map { column: Map.Entry<ColumnId, AirbyteType> ->
|
||||
java.lang.String.join(" ", column.key.name(QUOTE), toDialectType(column.value).name)
|
||||
}
|
||||
.joinToString(",\n")
|
||||
.collect(Collectors.joining(",\n"))
|
||||
}
|
||||
|
||||
override fun prepareTablesForSoftReset(
|
||||
stream: DestinationStream,
|
||||
tableNames: TableNames,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
): Sql {
|
||||
override fun prepareTablesForSoftReset(stream: StreamConfig): Sql {
|
||||
// Bigquery can't run DDL in a transaction, so these are separate transactions.
|
||||
return Sql.concat(
|
||||
// If a previous sync failed to delete the soft reset temp table (unclear why
|
||||
// this happens),
|
||||
// AND this sync is trying to change the clustering config, then we need to manually
|
||||
// drop the soft
|
||||
// reset temp table.
|
||||
// Even though we're using CREATE OR REPLACE TABLE, bigquery will still complain
|
||||
// about the
|
||||
// clustering config being changed.
|
||||
// So we explicitly drop the soft reset temp table first.
|
||||
dropTableIfExists(tableNames.finalTableName!!, TableNames.SOFT_RESET_SUFFIX),
|
||||
createFinalTable(
|
||||
stream,
|
||||
tableNames.finalTableName!!,
|
||||
columnNameMapping,
|
||||
TableNames.SOFT_RESET_SUFFIX,
|
||||
true
|
||||
),
|
||||
clearLoadedAt(stream, tableNames.rawTableName!!)
|
||||
)
|
||||
return Sql
|
||||
.concat( // If a previous sync failed to delete the soft reset temp table (unclear why
|
||||
// this happens),
|
||||
// AND this sync is trying to change the clustering config, then we need to manually
|
||||
// drop the soft
|
||||
// reset temp table.
|
||||
// Even though we're using CREATE OR REPLACE TABLE, bigquery will still complain
|
||||
// about the
|
||||
// clustering config being changed.
|
||||
// So we explicitly drop the soft reset temp table first.
|
||||
dropTableIfExists(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX),
|
||||
createTable(stream, TyperDeduperUtil.SOFT_RESET_SUFFIX, true),
|
||||
clearLoadedAt(stream.id)
|
||||
)
|
||||
}
|
||||
|
||||
private fun dropTableIfExists(
|
||||
finalTableName: TableName,
|
||||
suffix: String,
|
||||
): Sql {
|
||||
val tableId = finalTableName.toPrettyString(QUOTE, suffix)
|
||||
private fun dropTableIfExists(stream: StreamConfig, suffix: String): Sql {
|
||||
val tableId = stream.id.finalTableId(QUOTE, suffix)
|
||||
return Sql.of("""DROP TABLE IF EXISTS `$projectId`.$tableId;""")
|
||||
}
|
||||
|
||||
override fun clearLoadedAt(stream: DestinationStream, rawTableName: TableName): Sql {
|
||||
val rawTableId = rawTableName.toPrettyString(QUOTE)
|
||||
override fun clearLoadedAt(streamId: StreamId): Sql {
|
||||
val rawTableId = streamId.rawTableId(QUOTE)
|
||||
return Sql.of(
|
||||
"""UPDATE `$projectId`.$rawTableId SET _airbyte_loaded_at = NULL WHERE 1=1;"""
|
||||
)
|
||||
}
|
||||
|
||||
override fun updateFinalTable(
|
||||
stream: DestinationStream,
|
||||
tableNames: TableNames,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
finalTableSuffix: String,
|
||||
maxProcessedTimestamp: Instant?,
|
||||
useExpensiveSaferCasting: Boolean,
|
||||
override fun updateTable(
|
||||
stream: StreamConfig,
|
||||
finalSuffix: String,
|
||||
minRawTimestamp: Optional<Instant>,
|
||||
useExpensiveSaferCasting: Boolean
|
||||
): Sql {
|
||||
val handleNewRecords =
|
||||
if (stream.importType is Dedupe) {
|
||||
upsertNewRecords(
|
||||
stream,
|
||||
tableNames,
|
||||
columnNameMapping,
|
||||
finalTableSuffix,
|
||||
useExpensiveSaferCasting,
|
||||
maxProcessedTimestamp
|
||||
)
|
||||
if (stream.postImportAction == ImportType.DEDUPE) {
|
||||
upsertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp)
|
||||
} else {
|
||||
insertNewRecords(
|
||||
stream,
|
||||
tableNames,
|
||||
columnNameMapping,
|
||||
finalTableSuffix,
|
||||
useExpensiveSaferCasting,
|
||||
maxProcessedTimestamp
|
||||
)
|
||||
insertNewRecords(stream, finalSuffix, useExpensiveSaferCasting, minRawTimestamp)
|
||||
}
|
||||
val commitRawTable = commitRawTable(tableNames.rawTableName!!, maxProcessedTimestamp)
|
||||
val commitRawTable = commitRawTable(stream.id, minRawTimestamp)
|
||||
|
||||
return Sql.transactionally(handleNewRecords, commitRawTable)
|
||||
}
|
||||
|
||||
private fun insertNewRecords(
|
||||
stream: DestinationStream,
|
||||
tableNames: TableNames,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
stream: StreamConfig,
|
||||
finalSuffix: String,
|
||||
forceSafeCasting: Boolean,
|
||||
minRawTimestamp: Instant?,
|
||||
minRawTimestamp: Optional<Instant>
|
||||
): String {
|
||||
val columnList: String =
|
||||
stream.schema
|
||||
.asColumns()
|
||||
.keys
|
||||
stream.columns.keys
|
||||
.stream()
|
||||
.map { fieldName ->
|
||||
val columnName = columnNameMapping[fieldName]!!
|
||||
"`$columnName`,"
|
||||
}
|
||||
.map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," }
|
||||
.collect(Collectors.joining("\n"))
|
||||
val extractNewRawRecords =
|
||||
extractNewRawRecords(
|
||||
stream,
|
||||
tableNames,
|
||||
columnNameMapping,
|
||||
forceSafeCasting,
|
||||
minRawTimestamp
|
||||
)
|
||||
val finalTableId = tableNames.finalTableName!!.toPrettyString(QUOTE, finalSuffix)
|
||||
val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp)
|
||||
val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix)
|
||||
|
||||
return """
|
||||
INSERT INTO `$projectId`.$finalTableId
|
||||
@@ -292,45 +239,37 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
|
||||
private fun upsertNewRecords(
|
||||
stream: DestinationStream,
|
||||
tableNames: TableNames,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
stream: StreamConfig,
|
||||
finalSuffix: String,
|
||||
forceSafeCasting: Boolean,
|
||||
minRawTimestamp: Instant?,
|
||||
minRawTimestamp: Optional<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))"""
|
||||
}
|
||||
stream.primaryKey
|
||||
.stream()
|
||||
.map { pk: ColumnId ->
|
||||
val quotedPk = pk.name(QUOTE)
|
||||
("""(target_table.$quotedPk = new_record.$quotedPk OR (target_table.$quotedPk IS NULL AND new_record.$quotedPk IS NULL))""")
|
||||
}
|
||||
.collect(Collectors.joining(" AND "))
|
||||
|
||||
val columnList: String =
|
||||
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
|
||||
val columnName = columnNameMapping[fieldName]!!
|
||||
"`$columnName`,"
|
||||
}
|
||||
stream.columns.keys
|
||||
.stream()
|
||||
.map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," }
|
||||
.collect(Collectors.joining("\n"))
|
||||
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
|
||||
)
|
||||
stream.columns.keys
|
||||
.stream()
|
||||
.map { quotedColumnId: ColumnId ->
|
||||
"new_record." + quotedColumnId.name(QUOTE) + ","
|
||||
}
|
||||
.collect(Collectors.joining("\n"))
|
||||
val extractNewRawRecords = extractNewRawRecords(stream, forceSafeCasting, minRawTimestamp)
|
||||
|
||||
val cursorComparison: String
|
||||
if (importType.cursor.isNotEmpty()) {
|
||||
val cursorFieldName = importType.cursor.first()
|
||||
val cursorColumnName = columnNameMapping[cursorFieldName]!!
|
||||
val cursor = "`$cursorColumnName`"
|
||||
if (stream.cursor.isPresent) {
|
||||
val cursor = stream.cursor.get().name(QUOTE)
|
||||
// Build a condition for "new_record is more recent than target_table":
|
||||
cursorComparison = // First, compare the cursors.
|
||||
("""
|
||||
@@ -349,7 +288,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
|
||||
val cdcDeleteClause: String
|
||||
val cdcSkipInsertClause: String
|
||||
if (stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN)) {
|
||||
if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) {
|
||||
// Execute CDC deletions if there's already a record
|
||||
cdcDeleteClause =
|
||||
"WHEN MATCHED AND new_record._ab_cdc_deleted_at IS NOT NULL AND $cursorComparison THEN DELETE"
|
||||
@@ -363,11 +302,14 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
|
||||
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)
|
||||
stream.columns.keys
|
||||
.stream()
|
||||
.map { airbyteType: ColumnId ->
|
||||
val column = airbyteType.name(QUOTE)
|
||||
"$column = new_record.$column,"
|
||||
}
|
||||
.collect(Collectors.joining("\n"))
|
||||
val finalTableId = stream.id.finalTableId(QUOTE, finalSuffix)
|
||||
|
||||
return """
|
||||
MERGE `$projectId`.$finalTableId target_table
|
||||
@@ -406,29 +348,27 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
* dedupes the records (since we only need the most-recent record to upsert).
|
||||
*/
|
||||
private fun extractNewRawRecords(
|
||||
stream: DestinationStream,
|
||||
tableNames: TableNames,
|
||||
columnNameMapping: ColumnNameMapping,
|
||||
stream: StreamConfig,
|
||||
forceSafeCasting: Boolean,
|
||||
minRawTimestamp: Instant?,
|
||||
minRawTimestamp: Optional<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`,"
|
||||
stream.columns.entries
|
||||
.stream()
|
||||
.map { col: Map.Entry<ColumnId, AirbyteType> ->
|
||||
val extractAndCast = extractAndCast(col.key, col.value, forceSafeCasting)
|
||||
val columnName = col.key.name(QUOTE)
|
||||
"""$extractAndCast as $columnName,"""
|
||||
}
|
||||
.joinToString("\n")
|
||||
.collect(Collectors.joining("\n"))
|
||||
val columnErrors =
|
||||
if (forceSafeCasting) {
|
||||
"[" +
|
||||
stream.schema
|
||||
.asColumns()
|
||||
.map { (fieldName, type) ->
|
||||
val rawColName = escapeColumnNameForJsonPath(fieldName)
|
||||
val jsonExtract = extractAndCast(fieldName, type.type, true)
|
||||
stream.columns.entries
|
||||
.stream()
|
||||
.map { col: Map.Entry<ColumnId, AirbyteType> ->
|
||||
val rawColName = escapeColumnNameForJsonPath(col.key.originalName)
|
||||
val jsonExtract = extractAndCast(col.key, col.value, true)
|
||||
// Explicitly parse json here. This is safe because
|
||||
// we're not using the actual value anywhere,
|
||||
// and necessary because json_query
|
||||
@@ -442,7 +382,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
END
|
||||
""".trimIndent()
|
||||
}
|
||||
.joinToString(",\n") +
|
||||
.collect(Collectors.joining(",\n")) +
|
||||
"]"
|
||||
} else {
|
||||
// We're not safe casting, so any error should throw an exception and trigger the
|
||||
@@ -451,15 +391,14 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
|
||||
val columnList: String =
|
||||
stream.schema.asColumns().keys.joinToString("\n") { fieldName ->
|
||||
val columnName = columnNameMapping[fieldName]!!
|
||||
"`$columnName`,"
|
||||
}
|
||||
stream.columns.keys
|
||||
.stream()
|
||||
.map { quotedColumnId: ColumnId -> quotedColumnId.name(QUOTE) + "," }
|
||||
.collect(Collectors.joining("\n"))
|
||||
val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp)
|
||||
|
||||
val rawTableId = tableNames.rawTableName!!.toPrettyString(QUOTE)
|
||||
if (stream.importType is Dedupe) {
|
||||
val importType = stream.importType as Dedupe
|
||||
val rawTableId = stream.id.rawTableId(QUOTE)
|
||||
if (stream.postImportAction == ImportType.DEDUPE) {
|
||||
// When deduping, we need to dedup the raw records. Note the row_number() invocation in
|
||||
// the SQL
|
||||
// statement. Do the same extract+cast CTE + airbyte_meta construction as in non-dedup
|
||||
@@ -472,7 +411,7 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
// out-of-order records.
|
||||
|
||||
var cdcConditionalOrIncludeStatement = ""
|
||||
if (stream.schema.asColumns().containsKey(CDC_DELETED_AT_COLUMN)) {
|
||||
if (stream.columns.containsKey(CDC_DELETED_AT_COLUMN)) {
|
||||
cdcConditionalOrIncludeStatement =
|
||||
"""
|
||||
OR (
|
||||
@@ -483,21 +422,14 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
|
||||
val pkList =
|
||||
importType.primaryKey.joinToString(",") { fieldName ->
|
||||
val columnName = columnNameMapping[fieldName.first()]!!
|
||||
"`$columnName`"
|
||||
}
|
||||
stream.primaryKey
|
||||
.stream()
|
||||
.map { columnId: ColumnId -> columnId.name(QUOTE) }
|
||||
.collect(Collectors.joining(","))
|
||||
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}"
|
||||
)
|
||||
}
|
||||
stream.cursor
|
||||
.map { cursorId: ColumnId -> cursorId.name(QUOTE) + " DESC NULLS LAST," }
|
||||
.orElse("")
|
||||
|
||||
return """
|
||||
WITH intermediate_data AS (
|
||||
@@ -578,8 +510,8 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
fun commitRawTable(rawTableName: TableName, minRawTimestamp: Instant?): String {
|
||||
val rawTableId = rawTableName.toPrettyString(QUOTE)
|
||||
fun commitRawTable(id: StreamId, minRawTimestamp: Optional<Instant>): String {
|
||||
val rawTableId = id.rawTableId(QUOTE)
|
||||
val extractedAtCondition = buildExtractedAtCondition(minRawTimestamp)
|
||||
return """
|
||||
UPDATE `$projectId`.$rawTableId
|
||||
@@ -590,16 +522,56 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
""".trimIndent()
|
||||
}
|
||||
|
||||
override fun overwriteFinalTable(
|
||||
stream: DestinationStream,
|
||||
finalTableName: TableName,
|
||||
finalTableSuffix: String,
|
||||
): Sql {
|
||||
val finalTableId = finalTableName.toPrettyString(QUOTE)
|
||||
val tempFinalTableId = finalTableName.toPrettyString(QUOTE, finalTableSuffix)
|
||||
override fun overwriteFinalTable(stream: StreamId, finalSuffix: String): Sql {
|
||||
val finalTableId = stream.finalTableId(QUOTE)
|
||||
val tempFinalTableId = stream.finalTableId(QUOTE, finalSuffix)
|
||||
val realFinalTableName = stream.finalName(QUOTE)
|
||||
return Sql.separately(
|
||||
"DROP TABLE IF EXISTS `$projectId`.$finalTableId;",
|
||||
"ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO `${finalTableName.name}`;"
|
||||
"ALTER TABLE `$projectId`.$tempFinalTableId RENAME TO $realFinalTableName;"
|
||||
)
|
||||
}
|
||||
|
||||
private fun wrapAndQuote(namespace: String, tableName: String): String {
|
||||
return Stream.of(namespace, tableName)
|
||||
.map { part: String? -> StringUtils.wrap(part, QUOTE) }
|
||||
.collect(Collectors.joining("."))
|
||||
}
|
||||
|
||||
override fun createSchema(schema: String): Sql {
|
||||
val projectId = StringUtils.wrap(projectId, QUOTE)
|
||||
val quotedSchema = StringUtils.wrap(schema, QUOTE)
|
||||
return Sql.of(
|
||||
"""CREATE SCHEMA IF NOT EXISTS $projectId.$quotedSchema OPTIONS(location="$datasetLocation");"""
|
||||
)
|
||||
}
|
||||
|
||||
override fun migrateFromV1toV2(streamId: StreamId, namespace: String, tableName: String): Sql {
|
||||
val v2RawTable = streamId.rawTableId(QUOTE)
|
||||
val v1RawTable = wrapAndQuote(namespace, tableName)
|
||||
return Sql.of(
|
||||
"""
|
||||
CREATE OR REPLACE TABLE `$projectId`.$v2RawTable (
|
||||
_airbyte_raw_id STRING,
|
||||
_airbyte_data STRING,
|
||||
_airbyte_extracted_at TIMESTAMP,
|
||||
_airbyte_loaded_at TIMESTAMP,
|
||||
_airbyte_meta STRING,
|
||||
_airbyte_generation_id INTEGER
|
||||
)
|
||||
PARTITION BY DATE(_airbyte_extracted_at)
|
||||
CLUSTER BY _airbyte_extracted_at
|
||||
AS (
|
||||
SELECT
|
||||
_airbyte_ab_id AS _airbyte_raw_id,
|
||||
_airbyte_data AS _airbyte_data,
|
||||
_airbyte_emitted_at AS _airbyte_extracted_at,
|
||||
CAST(NULL AS TIMESTAMP) AS _airbyte_loaded_at,
|
||||
'{"sync_id": 0, "changes": []}' AS _airbyte_meta,
|
||||
0 as _airbyte_generation_id
|
||||
FROM `$projectId`.$v1RawTable
|
||||
);
|
||||
""".trimIndent()
|
||||
)
|
||||
}
|
||||
|
||||
@@ -634,55 +606,71 @@ class BigQuerySqlGenerator(private val projectId: String?, private val datasetLo
|
||||
|
||||
companion object {
|
||||
const val QUOTE: String = "`"
|
||||
val nameTransformer = BigQuerySQLNameTransformer()
|
||||
private 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
|
||||
@JvmStatic
|
||||
fun toDialectType(type: AirbyteType): StandardSQLTypeName {
|
||||
// switch pattern-matching is still in preview at language level 17 :(
|
||||
if (type is AirbyteProtocolType) {
|
||||
return toDialectType(type)
|
||||
} else if (type is Struct) {
|
||||
return StandardSQLTypeName.JSON
|
||||
} else if (type is Array) {
|
||||
return StandardSQLTypeName.JSON
|
||||
} else if (type is UnsupportedOneOf) {
|
||||
return StandardSQLTypeName.JSON
|
||||
} else if (type is Union) {
|
||||
val typeWithPrecedence: AirbyteType = type.chooseType()
|
||||
val dialectType: StandardSQLTypeName
|
||||
if ((typeWithPrecedence is Struct) || (typeWithPrecedence is Array)) {
|
||||
dialectType = StandardSQLTypeName.JSON
|
||||
} else {
|
||||
dialectType = toDialectType(typeWithPrecedence as AirbyteProtocolType)
|
||||
}
|
||||
return dialectType
|
||||
}
|
||||
|
||||
fun clusteringColumns(
|
||||
stream: DestinationStream,
|
||||
columnNameMapping: ColumnNameMapping
|
||||
): List<String> {
|
||||
// Literally impossible; AirbyteType is a sealed interface.
|
||||
throw IllegalArgumentException("Unsupported AirbyteType: $type")
|
||||
}
|
||||
|
||||
// TODO maybe make this a BiMap and elevate this method and its inverse
|
||||
// (toDestinationSQLType?) to
|
||||
// the SQLGenerator?
|
||||
fun toDialectType(airbyteProtocolType: AirbyteProtocolType): StandardSQLTypeName {
|
||||
return when (airbyteProtocolType) {
|
||||
AirbyteProtocolType.STRING,
|
||||
AirbyteProtocolType.TIME_WITH_TIMEZONE -> StandardSQLTypeName.STRING
|
||||
AirbyteProtocolType.NUMBER -> StandardSQLTypeName.NUMERIC
|
||||
AirbyteProtocolType.INTEGER -> StandardSQLTypeName.INT64
|
||||
AirbyteProtocolType.BOOLEAN -> StandardSQLTypeName.BOOL
|
||||
AirbyteProtocolType.TIMESTAMP_WITH_TIMEZONE -> StandardSQLTypeName.TIMESTAMP
|
||||
AirbyteProtocolType.TIMESTAMP_WITHOUT_TIMEZONE -> StandardSQLTypeName.DATETIME
|
||||
AirbyteProtocolType.TIME_WITHOUT_TIMEZONE -> StandardSQLTypeName.TIME
|
||||
AirbyteProtocolType.DATE -> StandardSQLTypeName.DATE
|
||||
AirbyteProtocolType.UNKNOWN -> StandardSQLTypeName.JSON
|
||||
}
|
||||
}
|
||||
|
||||
fun clusteringColumns(stream: StreamConfig): List<String> {
|
||||
val clusterColumns: MutableList<String> = ArrayList()
|
||||
if (stream.importType is Dedupe) {
|
||||
if (stream.postImportAction == ImportType.DEDUPE) {
|
||||
// We're doing de-duping, therefore we have a primary key.
|
||||
// Cluster on the first 3 PK columns since BigQuery only allows up to 4 clustering
|
||||
// columns,
|
||||
// and we're always clustering on _airbyte_extracted_at
|
||||
(stream.importType as Dedupe).primaryKey.stream().limit(3).forEach {
|
||||
pk: List<String> ->
|
||||
clusterColumns.add(columnNameMapping[pk.first()]!!)
|
||||
stream.primaryKey.stream().limit(3).forEach { columnId: ColumnId ->
|
||||
clusterColumns.add(columnId.name)
|
||||
}
|
||||
}
|
||||
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 buildExtractedAtCondition(minRawTimestamp: Optional<Instant>): String {
|
||||
return minRawTimestamp
|
||||
.map { ts: Instant -> " AND _airbyte_extracted_at > '$ts'" }
|
||||
.orElse("")
|
||||
}
|
||||
|
||||
private fun cast(content: String, asType: String, useSafeCast: Boolean): String {
|
||||
|
||||
@@ -0,0 +1,60 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.google.cloud.bigquery.*
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.BaseDestinationV1V2Migrator
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.CollectionUtils.containsAllIgnoreCase
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.NamespacedTableName
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.destination.bigquery.BigQuerySQLNameTransformer
|
||||
import java.util.*
|
||||
import java.util.stream.Collectors
|
||||
|
||||
class BigQueryV1V2Migrator(
|
||||
private val bq: BigQuery,
|
||||
private val nameTransformer: BigQuerySQLNameTransformer
|
||||
) : BaseDestinationV1V2Migrator<TableDefinition>() {
|
||||
@SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE")
|
||||
override fun doesAirbyteInternalNamespaceExist(streamConfig: StreamConfig?): Boolean {
|
||||
val dataset = bq.getDataset(streamConfig!!.id.rawNamespace)
|
||||
return dataset != null && dataset.exists()
|
||||
}
|
||||
|
||||
override fun getTableIfExists(
|
||||
namespace: String?,
|
||||
tableName: String?
|
||||
): Optional<TableDefinition> {
|
||||
val table = bq.getTable(TableId.of(namespace, tableName))
|
||||
return if (table != null && table.exists()) Optional.of(table.getDefinition())
|
||||
else Optional.empty()
|
||||
}
|
||||
|
||||
override fun schemaMatchesExpectation(
|
||||
existingTable: TableDefinition,
|
||||
columns: Collection<String>
|
||||
): Boolean {
|
||||
val existingSchemaColumns =
|
||||
Optional.ofNullable(existingTable.schema)
|
||||
.map { schema: Schema ->
|
||||
schema.fields
|
||||
.stream()
|
||||
.map { obj: Field -> obj.name }
|
||||
.collect(Collectors.toSet())
|
||||
}
|
||||
.orElse(emptySet())
|
||||
|
||||
return !existingSchemaColumns.isEmpty() &&
|
||||
containsAllIgnoreCase(columns, existingSchemaColumns)
|
||||
}
|
||||
|
||||
@Suppress("deprecation")
|
||||
override fun convertToV1RawName(streamConfig: StreamConfig): NamespacedTableName {
|
||||
return NamespacedTableName(
|
||||
nameTransformer.getNamespace(streamConfig.id.originalNamespace),
|
||||
nameTransformer.getRawTableName(streamConfig.id.originalName)
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -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}")
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -1,54 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.orchestration.db.ColumnNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.FinalTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.RawTableNameGenerator
|
||||
import io.airbyte.cdk.load.orchestration.db.TableName
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingUtil
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator.Companion.nameTransformer
|
||||
import java.util.Locale
|
||||
import javax.inject.Singleton
|
||||
|
||||
@Singleton
|
||||
class BigqueryRawTableNameGenerator(val config: BigqueryConfiguration) : RawTableNameGenerator {
|
||||
override fun getTableName(streamDescriptor: DestinationStream.Descriptor) =
|
||||
TableName(
|
||||
nameTransformer.getNamespace(config.rawTableDataset),
|
||||
nameTransformer.convertStreamName(
|
||||
TypingDedupingUtil.concatenateRawTableName(
|
||||
streamDescriptor.namespace ?: config.datasetId,
|
||||
streamDescriptor.name,
|
||||
)
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigqueryFinalTableNameGenerator(val config: BigqueryConfiguration) : FinalTableNameGenerator {
|
||||
override fun getTableName(streamDescriptor: DestinationStream.Descriptor) =
|
||||
TableName(
|
||||
nameTransformer.getNamespace(streamDescriptor.namespace ?: config.datasetId),
|
||||
nameTransformer.convertStreamName(streamDescriptor.name),
|
||||
)
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigqueryColumnNameGenerator : ColumnNameGenerator {
|
||||
override fun getColumnName(column: String): ColumnNameGenerator.ColumnName {
|
||||
return ColumnNameGenerator.ColumnName(
|
||||
nameTransformer.convertStreamName(column),
|
||||
// Bigquery columns are case-insensitive, so do all our validation on the
|
||||
// lowercased name
|
||||
nameTransformer.convertStreamName(column.lowercase(Locale.getDefault())),
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
fun TableName.toTableId(): TableId = TableId.of(this.namespace, this.name)
|
||||
@@ -1,115 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.write
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import com.google.cloud.bigquery.TableResult
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.orchestration.db.TableName
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingRawTableOperations
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
class BigqueryRawTableOperations(private val bigquery: BigQuery) :
|
||||
TypingDedupingRawTableOperations {
|
||||
override fun prepareRawTable(rawTableName: TableName, suffix: String, replace: Boolean) {
|
||||
// Prepare staging table. For overwrite, it does drop-create so we can skip explicit create.
|
||||
if (replace) {
|
||||
truncateStagingTable(rawTableName, suffix)
|
||||
} else {
|
||||
createStagingTable(rawTableName, suffix)
|
||||
}
|
||||
}
|
||||
|
||||
override fun overwriteRawTable(rawTableName: TableName, suffix: String) {
|
||||
if (suffix == "") {
|
||||
throw IllegalArgumentException("Cannot overwrite raw table with empty suffix")
|
||||
}
|
||||
bigquery.delete(tableId(rawTableName, ""))
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"""ALTER TABLE `${rawTableName.namespace}`.`${rawTableName.name}$suffix` RENAME TO `${rawTableName.name}`"""
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
override fun transferFromTempRawTable(rawTableName: TableName, suffix: String) {
|
||||
if (suffix == "") {
|
||||
throw IllegalArgumentException(
|
||||
"Cannot transfer records from temp raw table with empty suffix"
|
||||
)
|
||||
}
|
||||
// TODO figure out how to make this work
|
||||
// something about incompatible partitioning spec (probably b/c we're copying from a temp
|
||||
// table partitioned on generation ID into an old real raw table partitioned on
|
||||
// extracted_at)
|
||||
val tempRawTable = tableId(rawTableName, suffix)
|
||||
// val jobConf =
|
||||
// CopyJobConfiguration.newBuilder(tableId(streamId, ""), tempRawTable)
|
||||
// .setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
|
||||
// .build()
|
||||
// val job = bigquery.create(JobInfo.of(jobConf))
|
||||
// BigQueryUtils.waitForJobFinish(job)
|
||||
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"""
|
||||
INSERT INTO `${rawTableName.namespace}`.`${rawTableName.name}`
|
||||
SELECT * FROM `${rawTableName.namespace}`.`${rawTableName.name}$suffix`
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
bigquery.delete(tempRawTable)
|
||||
}
|
||||
|
||||
override fun getRawTableGeneration(rawTableName: TableName, suffix: String): Long? {
|
||||
val result: TableResult =
|
||||
bigquery.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT _airbyte_generation_id FROM ${rawTableName.namespace}.${rawTableName.name}$suffix LIMIT 1"
|
||||
),
|
||||
)
|
||||
if (result.totalRows == 0L) {
|
||||
return null
|
||||
}
|
||||
val value = result.iterateAll().first().get(Meta.COLUMN_NAME_AB_GENERATION_ID)
|
||||
return if (value == null || value.isNull) {
|
||||
0
|
||||
} else {
|
||||
value.longValue
|
||||
}
|
||||
}
|
||||
|
||||
private fun createStagingTable(rawTableName: TableName, suffix: String) {
|
||||
BigQueryUtils.createPartitionedTableIfNotExists(
|
||||
bigquery,
|
||||
tableId(rawTableName, suffix),
|
||||
BigQueryRecordFormatter.SCHEMA_V2,
|
||||
)
|
||||
}
|
||||
|
||||
private fun dropStagingTable(rawTableName: TableName, suffix: String) {
|
||||
bigquery.delete(tableId(rawTableName, suffix))
|
||||
}
|
||||
|
||||
/**
|
||||
* "Truncates" table, this is a workaround to the issue with TRUNCATE TABLE in BigQuery where
|
||||
* the table's partition filter must be turned off to truncate. Since deleting a table is a free
|
||||
* operation this option re-uses functions that already exist
|
||||
*/
|
||||
private fun truncateStagingTable(rawTableName: TableName, suffix: String) {
|
||||
logger.info { "Truncating raw table ${tableId(rawTableName, suffix)}" }
|
||||
dropStagingTable(rawTableName, suffix)
|
||||
createStagingTable(rawTableName, suffix)
|
||||
}
|
||||
|
||||
private fun tableId(rawTableName: TableName, suffix: String = ""): TableId =
|
||||
TableId.of(rawTableName.namespace, rawTableName.name + suffix)
|
||||
}
|
||||
@@ -1,112 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.write.bulk_loader
|
||||
|
||||
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.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.legacy_typing_deduping.TableCatalogByDescriptor
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.cdk.load.write.db.BulkLoader
|
||||
import io.airbyte.cdk.load.write.db.BulkLoaderFactory
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
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.micronaut.context.annotation.Requires
|
||||
import io.micronaut.context.condition.Condition
|
||||
import io.micronaut.context.condition.ConditionContext
|
||||
import jakarta.inject.Singleton
|
||||
|
||||
class BigQueryBulkLoader(
|
||||
private val storageClient: GcsClient,
|
||||
private val bigQueryClient: BigQuery,
|
||||
private val bigQueryConfiguration: BigqueryConfiguration,
|
||||
private val rawTableName: TableName,
|
||||
private val rawTableSuffix: String,
|
||||
) : 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 =
|
||||
CsvOptions.newBuilder()
|
||||
.setSkipLeadingRows(1)
|
||||
.setAllowQuotedNewLines(true) // safe for long JSON strings
|
||||
.setAllowJaggedRows(true)
|
||||
.build()
|
||||
|
||||
val configuration =
|
||||
LoadJobConfiguration.builder(rawTableId, gcsUri)
|
||||
.setFormatOptions(csvOptions)
|
||||
.setSchema(BigQueryRecordFormatter.CSV_SCHEMA)
|
||||
.setWriteDisposition(JobInfo.WriteDisposition.WRITE_APPEND)
|
||||
.setJobTimeoutMs(600000L) // 10 min timeout
|
||||
.build()
|
||||
|
||||
val loadJob = bigQueryClient.create(JobInfo.of(configuration))
|
||||
|
||||
try {
|
||||
BigQueryUtils.waitForJobFinish(loadJob)
|
||||
} catch (e: Exception) {
|
||||
throw RuntimeException(
|
||||
"Failed to load CSV data from $gcsUri to table ${rawTableId.dataset}.${rawTableId.table}: ${e.message}",
|
||||
e
|
||||
)
|
||||
}
|
||||
|
||||
val loadingMethodPostProcessing =
|
||||
(bigQueryConfiguration.loadingMethod as GcsStagingConfiguration).filePostProcessing
|
||||
if (loadingMethodPostProcessing == GcsFilePostProcessing.DELETE) {
|
||||
storageClient.delete(remoteObject)
|
||||
}
|
||||
}
|
||||
|
||||
override fun close() {
|
||||
/* Do nothing */
|
||||
}
|
||||
}
|
||||
|
||||
class BigqueryConfiguredForBulkLoad : Condition {
|
||||
override fun matches(context: ConditionContext<*>): Boolean {
|
||||
val config = context.beanContext.getBean(BigqueryConfiguration::class.java)
|
||||
return config.loadingMethod is GcsStagingConfiguration
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
@Requires(condition = BigqueryConfiguredForBulkLoad::class)
|
||||
class BigQueryBulkLoaderFactory(
|
||||
private val names: TableCatalogByDescriptor,
|
||||
private val storageClient: GcsClient,
|
||||
private val bigQueryClient: BigQuery,
|
||||
private val bigQueryConfiguration: BigqueryConfiguration,
|
||||
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
) : BulkLoaderFactory<StreamKey, GcsBlob> {
|
||||
override val numPartWorkers: Int = 2
|
||||
override val numUploadWorkers: Int = 10
|
||||
override val maxNumConcurrentLoads: Int = 1
|
||||
|
||||
override val objectSizeBytes: Long = 200 * 1024 * 1024 // 200 MB
|
||||
override val partSizeBytes: Long = 10 * 1024 * 1024 // 10 MB
|
||||
override val maxMemoryRatioReservedForParts: Double = 0.6
|
||||
|
||||
override fun create(key: StreamKey, partition: Int): BulkLoader<GcsBlob> {
|
||||
return BigQueryBulkLoader(
|
||||
storageClient,
|
||||
bigQueryClient,
|
||||
bigQueryConfiguration,
|
||||
names[key.stream]!!.tableNames.rawTableName!!,
|
||||
streamStateStore.get(key.stream)!!.rawTableSuffix,
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -1,80 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.write.bulk_loader
|
||||
|
||||
import io.airbyte.cdk.load.command.aws.AWSAccessKeyConfiguration
|
||||
import io.airbyte.cdk.load.command.aws.AWSAccessKeyConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.aws.AWSArnRoleConfiguration
|
||||
import io.airbyte.cdk.load.command.aws.AWSArnRoleConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.gcs.GOOGLE_STORAGE_ENDPOINT
|
||||
import io.airbyte.cdk.load.command.gcs.GcsClientConfiguration
|
||||
import io.airbyte.cdk.load.command.gcs.GcsClientConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.gcs.GcsHmacKeyConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.CSVFormatConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageCompressionConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageCompressionConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageFormatConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageFormatConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStoragePathConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStoragePathConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageUploadConfiguration
|
||||
import io.airbyte.cdk.load.command.object_storage.ObjectStorageUploadConfigurationProvider
|
||||
import io.airbyte.cdk.load.command.s3.S3BucketConfiguration
|
||||
import io.airbyte.cdk.load.command.s3.S3BucketConfigurationProvider
|
||||
import io.airbyte.cdk.load.file.GZIPProcessor
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigqueryConfiguration
|
||||
import io.airbyte.integrations.destination.bigquery.spec.GcsStagingConfiguration
|
||||
import java.io.BufferedOutputStream
|
||||
|
||||
data class BigqueryBulkLoadConfiguration(
|
||||
val bigQueryConfiguration: BigqueryConfiguration,
|
||||
) :
|
||||
ObjectStoragePathConfigurationProvider,
|
||||
ObjectStorageFormatConfigurationProvider,
|
||||
ObjectStorageUploadConfigurationProvider,
|
||||
S3BucketConfigurationProvider,
|
||||
AWSAccessKeyConfigurationProvider,
|
||||
AWSArnRoleConfigurationProvider,
|
||||
GcsClientConfigurationProvider,
|
||||
ObjectStorageCompressionConfigurationProvider<BufferedOutputStream> {
|
||||
override val objectStoragePathConfiguration: ObjectStoragePathConfiguration
|
||||
override val objectStorageFormatConfiguration: ObjectStorageFormatConfiguration =
|
||||
CSVFormatConfiguration()
|
||||
override val objectStorageUploadConfiguration: ObjectStorageUploadConfiguration =
|
||||
ObjectStorageUploadConfiguration()
|
||||
override val s3BucketConfiguration: S3BucketConfiguration
|
||||
override val awsAccessKeyConfiguration: AWSAccessKeyConfiguration
|
||||
override val awsArnRoleConfiguration: AWSArnRoleConfiguration = AWSArnRoleConfiguration(null)
|
||||
override val gcsClientConfiguration: GcsClientConfiguration =
|
||||
(bigQueryConfiguration.loadingMethod as GcsStagingConfiguration).gcsClientConfig
|
||||
override val objectStorageCompressionConfiguration =
|
||||
ObjectStorageCompressionConfiguration(GZIPProcessor)
|
||||
|
||||
init {
|
||||
bigQueryConfiguration.loadingMethod as GcsStagingConfiguration
|
||||
s3BucketConfiguration =
|
||||
S3BucketConfiguration(
|
||||
s3BucketName = bigQueryConfiguration.loadingMethod.gcsClientConfig.gcsBucketName,
|
||||
s3BucketRegion = bigQueryConfiguration.loadingMethod.gcsClientConfig.region,
|
||||
s3Endpoint = GOOGLE_STORAGE_ENDPOINT,
|
||||
)
|
||||
val credentials =
|
||||
bigQueryConfiguration.loadingMethod.gcsClientConfig.credential
|
||||
as GcsHmacKeyConfiguration
|
||||
awsAccessKeyConfiguration =
|
||||
AWSAccessKeyConfiguration(
|
||||
accessKeyId = credentials.accessKeyId,
|
||||
secretAccessKey = credentials.secretAccessKey
|
||||
)
|
||||
|
||||
objectStoragePathConfiguration =
|
||||
ObjectStoragePathConfiguration(
|
||||
prefix = bigQueryConfiguration.loadingMethod.gcsClientConfig.path,
|
||||
pathPattern =
|
||||
"\${NAMESPACE}/\${STREAM_NAME}/\${YEAR}/\${MONTH}/\${DAY}/\${HOUR}/\${UUID}",
|
||||
fileNamePattern = "{date}_{timestamp}_{part_number}{format_extension}",
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -1,43 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
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.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 jakarta.inject.Singleton
|
||||
import java.io.OutputStream
|
||||
|
||||
class BigQueryObjectStorageFormattingWriter(
|
||||
private val csvFormattingWriter: CSVFormattingWriter,
|
||||
) : ObjectStorageFormattingWriter by csvFormattingWriter {
|
||||
|
||||
override fun accept(record: DestinationRecordRaw) {
|
||||
csvFormattingWriter.accept(record)
|
||||
}
|
||||
}
|
||||
|
||||
@Singleton
|
||||
class BigQueryObjectStorageFormattingWriterFactory(
|
||||
private val formatConfigProvider: ObjectStorageFormatConfigurationProvider,
|
||||
) : ObjectStorageFormattingWriterFactory {
|
||||
override fun create(
|
||||
stream: DestinationStream,
|
||||
outputStream: OutputStream
|
||||
): ObjectStorageFormattingWriter {
|
||||
val flatten = formatConfigProvider.objectStorageFormatConfiguration.rootLevelFlattening
|
||||
return BigQueryObjectStorageFormattingWriter(
|
||||
CSVFormattingWriter(
|
||||
stream,
|
||||
outputStream,
|
||||
rootLevelFlattening = flatten,
|
||||
extractedAtAsTimestampWithTimezone = true,
|
||||
),
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -1,167 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.write.standard_insert
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.FormatOptions
|
||||
import com.google.cloud.bigquery.JobId
|
||||
import com.google.cloud.bigquery.JobInfo
|
||||
import com.google.cloud.bigquery.TableDataWriteChannel
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import com.google.cloud.bigquery.WriteChannelConfiguration
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.cdk.ConfigErrorException
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.message.DestinationRecordRaw
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TableCatalogByDescriptor
|
||||
import io.airbyte.cdk.load.orchestration.db.legacy_typing_deduping.TypingDedupingExecutionConfig
|
||||
import io.airbyte.cdk.load.write.DirectLoader
|
||||
import io.airbyte.cdk.load.write.DirectLoaderFactory
|
||||
import io.airbyte.cdk.load.write.StreamStateStore
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BatchedStandardInsertConfiguration
|
||||
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.micronaut.context.annotation.Requires
|
||||
import io.micronaut.context.condition.Condition
|
||||
import io.micronaut.context.condition.ConditionContext
|
||||
import jakarta.inject.Singleton
|
||||
import java.io.ByteArrayOutputStream
|
||||
import java.nio.ByteBuffer
|
||||
import java.nio.charset.StandardCharsets
|
||||
|
||||
class BigqueryBatchStandardInsertsLoader(
|
||||
private val bigquery: BigQuery,
|
||||
private val writeChannelConfiguration: WriteChannelConfiguration,
|
||||
private val job: JobId,
|
||||
) : 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
|
||||
// to writing to the writechannel directly.
|
||||
// invariant: either the buffer is nonnull, or the writer is initialized. They are never both
|
||||
// active at the same time.
|
||||
// bigquery sets daily limits on how many TableDataWriteChannel jobs you can run,
|
||||
// so we can't just flush+close a TableDataWriteChannel as soon as we reach 15MB.
|
||||
private var buffer: ByteArrayOutputStream? = ByteArrayOutputStream()
|
||||
private lateinit var writer: TableDataWriteChannel
|
||||
|
||||
@SuppressFBWarnings("RCN_REDUNDANT_NULLCHECK_OF_NONNULL_VALUE")
|
||||
override suspend fun accept(record: DestinationRecordRaw): DirectLoader.DirectLoadResult {
|
||||
val formattedRecord = recordFormatter.formatRecord(record)
|
||||
val byteArray =
|
||||
"$formattedRecord${System.lineSeparator()}".toByteArray(StandardCharsets.UTF_8)
|
||||
|
||||
if (this::writer.isInitialized) {
|
||||
writer.write(ByteBuffer.wrap(byteArray))
|
||||
} else {
|
||||
buffer!!.write(byteArray)
|
||||
// the default chunk size on the TableDataWriteChannel is 15MB,
|
||||
// so switch to writing to a real writechannel when we reach that size
|
||||
if (buffer!!.size() > 15 * 1024 * 1024) {
|
||||
switchToWriteChannel()
|
||||
}
|
||||
}
|
||||
|
||||
// rely on the CDK to tell us when to finish()
|
||||
return DirectLoader.Incomplete
|
||||
}
|
||||
|
||||
override suspend fun finish() {
|
||||
if (!this::writer.isInitialized) {
|
||||
switchToWriteChannel()
|
||||
}
|
||||
writer.close()
|
||||
BigQueryUtils.waitForJobFinish(writer.job)
|
||||
}
|
||||
|
||||
override fun close() {}
|
||||
|
||||
// Somehow spotbugs thinks that `writer.write(ByteBuffer.wrap(byteArray))` is a redundant null
|
||||
// check...
|
||||
@SuppressFBWarnings(value = ["RCN_REDUNDANT_NULLCHECK_WOULD_HAVE_BEEN_A_NPE"])
|
||||
private fun switchToWriteChannel() {
|
||||
writer =
|
||||
try {
|
||||
bigquery.writer(job, writeChannelConfiguration)
|
||||
} catch (e: BigQueryException) {
|
||||
if (e.code == HTTP_STATUS_CODE_FORBIDDEN || e.code == HTTP_STATUS_CODE_NOT_FOUND) {
|
||||
throw ConfigErrorException(CONFIG_ERROR_MSG + e)
|
||||
} else {
|
||||
throw BigQueryException(e.code, e.message)
|
||||
}
|
||||
}
|
||||
val byteArray = buffer!!.toByteArray()
|
||||
// please GC this object :)
|
||||
buffer = null
|
||||
writer.write(ByteBuffer.wrap(byteArray))
|
||||
}
|
||||
}
|
||||
|
||||
class BigqueryConfiguredForBatchStandardInserts : Condition {
|
||||
override fun matches(context: ConditionContext<*>): Boolean {
|
||||
val config = context.beanContext.getBean(BigqueryConfiguration::class.java)
|
||||
return config.loadingMethod is BatchedStandardInsertConfiguration
|
||||
}
|
||||
}
|
||||
|
||||
@Requires(condition = BigqueryConfiguredForBatchStandardInserts::class)
|
||||
@Singleton
|
||||
class BigqueryBatchStandardInsertsLoaderFactory(
|
||||
private val bigquery: BigQuery,
|
||||
private val config: BigqueryConfiguration,
|
||||
private val tableCatalog: TableCatalogByDescriptor,
|
||||
private val streamStateStore: StreamStateStore<TypingDedupingExecutionConfig>,
|
||||
) : 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)
|
||||
)
|
||||
.setCreateDisposition(JobInfo.CreateDisposition.CREATE_IF_NEEDED)
|
||||
.setSchema(BigQueryRecordFormatter.SCHEMA_V2)
|
||||
// new-line delimited json.
|
||||
.setFormatOptions(FormatOptions.json())
|
||||
.build()
|
||||
|
||||
val jobId =
|
||||
JobId.newBuilder()
|
||||
.setRandomJob()
|
||||
.setLocation(config.datasetLocation.region)
|
||||
.setProject(bigquery.options.projectId)
|
||||
.build()
|
||||
|
||||
return BigqueryBatchStandardInsertsLoader(
|
||||
bigquery,
|
||||
writeChannelConfiguration,
|
||||
jobId,
|
||||
)
|
||||
}
|
||||
|
||||
companion object {
|
||||
const val HTTP_STATUS_CODE_FORBIDDEN = 403
|
||||
const val HTTP_STATUS_CODE_NOT_FOUND = 404
|
||||
|
||||
val CONFIG_ERROR_MSG =
|
||||
"""
|
||||
|Failed to write to destination schema.
|
||||
| 1. Make sure you have all required permissions for writing to the schema.
|
||||
| 2. Make sure that the actual destination schema's location corresponds to the location provided in the connector's config.
|
||||
| 3. Try to change the "Destination schema" from "Mirror Source Structure" (if it's set) to the "Destination Default" option.
|
||||
|More details:
|
||||
|""".trimMargin()
|
||||
}
|
||||
}
|
||||
@@ -1,5 +0,0 @@
|
||||
airbyte:
|
||||
destination:
|
||||
core:
|
||||
types:
|
||||
unions: LEGACY
|
||||
@@ -0,0 +1,240 @@
|
||||
{
|
||||
"documentationUrl": "https://docs.airbyte.com/integrations/destinations/bigquery",
|
||||
"supportsIncremental": true,
|
||||
"supportsDBT": true,
|
||||
"supported_destination_sync_modes": ["overwrite", "append", "append_dedup"],
|
||||
"connectionSpecification": {
|
||||
"$schema": "http://json-schema.org/draft-07/schema#",
|
||||
"title": "BigQuery Destination Spec",
|
||||
"type": "object",
|
||||
"required": ["project_id", "dataset_location", "dataset_id"],
|
||||
"additionalProperties": true,
|
||||
"properties": {
|
||||
"project_id": {
|
||||
"type": "string",
|
||||
"description": "The GCP project ID for the project containing the target BigQuery dataset. Read more <a href=\"https://cloud.google.com/resource-manager/docs/creating-managing-projects#identifying_projects\">here</a>.",
|
||||
"title": "Project ID",
|
||||
"group": "connection",
|
||||
"order": 0
|
||||
},
|
||||
"dataset_location": {
|
||||
"type": "string",
|
||||
"description": "The location of the dataset. Warning: Changes made after creation will not be applied. Read more <a href=\"https://cloud.google.com/bigquery/docs/locations\">here</a>.",
|
||||
"title": "Dataset Location",
|
||||
"group": "connection",
|
||||
"order": 1,
|
||||
"enum": [
|
||||
"US",
|
||||
"EU",
|
||||
"asia-east1",
|
||||
"asia-east2",
|
||||
"asia-northeast1",
|
||||
"asia-northeast2",
|
||||
"asia-northeast3",
|
||||
"asia-south1",
|
||||
"asia-south2",
|
||||
"asia-southeast1",
|
||||
"asia-southeast2",
|
||||
"australia-southeast1",
|
||||
"australia-southeast2",
|
||||
"europe-central1",
|
||||
"europe-central2",
|
||||
"europe-north1",
|
||||
"europe-north2",
|
||||
"europe-southwest1",
|
||||
"europe-west1",
|
||||
"europe-west2",
|
||||
"europe-west3",
|
||||
"europe-west4",
|
||||
"europe-west6",
|
||||
"europe-west7",
|
||||
"europe-west8",
|
||||
"europe-west9",
|
||||
"europe-west12",
|
||||
"me-central1",
|
||||
"me-central2",
|
||||
"me-west1",
|
||||
"northamerica-northeast1",
|
||||
"northamerica-northeast2",
|
||||
"southamerica-east1",
|
||||
"southamerica-west1",
|
||||
"us-central1",
|
||||
"us-east1",
|
||||
"us-east2",
|
||||
"us-east3",
|
||||
"us-east4",
|
||||
"us-east5",
|
||||
"us-south1",
|
||||
"us-west1",
|
||||
"us-west2",
|
||||
"us-west3",
|
||||
"us-west4"
|
||||
]
|
||||
},
|
||||
"dataset_id": {
|
||||
"type": "string",
|
||||
"description": "The default BigQuery Dataset ID that tables are replicated to if the source does not specify a namespace. Read more <a href=\"https://cloud.google.com/bigquery/docs/datasets#create-dataset\">here</a>.",
|
||||
"title": "Default Dataset ID",
|
||||
"group": "connection",
|
||||
"order": 2
|
||||
},
|
||||
"loading_method": {
|
||||
"type": "object",
|
||||
"title": "Loading Method",
|
||||
"description": "The way data will be uploaded to BigQuery.",
|
||||
"display_type": "radio",
|
||||
"group": "connection",
|
||||
"order": 3,
|
||||
"oneOf": [
|
||||
{
|
||||
"title": "Batched Standard Inserts",
|
||||
"required": ["method"],
|
||||
"description": "Direct loading using batched SQL INSERT statements. This method uses the BigQuery driver to convert large INSERT statements into file uploads automatically.",
|
||||
"properties": {
|
||||
"method": {
|
||||
"type": "string",
|
||||
"const": "Standard"
|
||||
}
|
||||
}
|
||||
},
|
||||
{
|
||||
"title": "GCS Staging",
|
||||
"description": "Writes large batches of records to a file, uploads the file to GCS, then uses COPY INTO to load your data into BigQuery.",
|
||||
"required": [
|
||||
"method",
|
||||
"gcs_bucket_name",
|
||||
"gcs_bucket_path",
|
||||
"credential"
|
||||
],
|
||||
"properties": {
|
||||
"method": {
|
||||
"type": "string",
|
||||
"const": "GCS Staging"
|
||||
},
|
||||
"credential": {
|
||||
"title": "Credential",
|
||||
"description": "An HMAC key is a type of credential and can be associated with a service account or a user account in Cloud Storage. Read more <a href=\"https://cloud.google.com/storage/docs/authentication/hmackeys\">here</a>.",
|
||||
"type": "object",
|
||||
"order": 1,
|
||||
"oneOf": [
|
||||
{
|
||||
"title": "HMAC key",
|
||||
"required": [
|
||||
"credential_type",
|
||||
"hmac_key_access_id",
|
||||
"hmac_key_secret"
|
||||
],
|
||||
"properties": {
|
||||
"credential_type": {
|
||||
"type": "string",
|
||||
"const": "HMAC_KEY",
|
||||
"order": 0
|
||||
},
|
||||
"hmac_key_access_id": {
|
||||
"type": "string",
|
||||
"description": "HMAC key access ID. When linked to a service account, this ID is 61 characters long; when linked to a user account, it is 24 characters long.",
|
||||
"title": "HMAC Key Access ID",
|
||||
"airbyte_secret": true,
|
||||
"examples": ["1234567890abcdefghij1234"],
|
||||
"order": 1
|
||||
},
|
||||
"hmac_key_secret": {
|
||||
"type": "string",
|
||||
"description": "The corresponding secret for the access ID. It is a 40-character base-64 encoded string.",
|
||||
"title": "HMAC Key Secret",
|
||||
"airbyte_secret": true,
|
||||
"examples": [
|
||||
"1234567890abcdefghij1234567890ABCDEFGHIJ"
|
||||
],
|
||||
"order": 2
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"gcs_bucket_name": {
|
||||
"title": "GCS Bucket Name",
|
||||
"type": "string",
|
||||
"description": "The name of the GCS bucket. Read more <a href=\"https://cloud.google.com/storage/docs/naming-buckets\">here</a>.",
|
||||
"examples": ["airbyte_sync"],
|
||||
"order": 2
|
||||
},
|
||||
"gcs_bucket_path": {
|
||||
"title": "GCS Bucket Path",
|
||||
"description": "Directory under the GCS bucket where data will be written.",
|
||||
"type": "string",
|
||||
"examples": ["data_sync/test"],
|
||||
"order": 3
|
||||
},
|
||||
"keep_files_in_gcs-bucket": {
|
||||
"type": "string",
|
||||
"description": "This upload method is supposed to temporary store records in GCS bucket. By this select you can chose if these records should be removed from GCS when migration has finished. The default \"Delete all tmp files from GCS\" value is used if not set explicitly.",
|
||||
"title": "GCS Tmp Files Afterward Processing",
|
||||
"default": "Delete all tmp files from GCS",
|
||||
"enum": [
|
||||
"Delete all tmp files from GCS",
|
||||
"Keep all tmp files in GCS"
|
||||
],
|
||||
"order": 4
|
||||
}
|
||||
}
|
||||
}
|
||||
]
|
||||
},
|
||||
"credentials_json": {
|
||||
"type": "string",
|
||||
"description": "The contents of the JSON service account key. Check out the <a href=\"https://docs.airbyte.com/integrations/destinations/bigquery#service-account-key\">docs</a> if you need help generating this key. Default credentials will be used if this field is left empty.",
|
||||
"title": "Service Account Key JSON (Required for cloud, optional for open-source)",
|
||||
"airbyte_secret": true,
|
||||
"group": "connection",
|
||||
"order": 4,
|
||||
"always_show": true
|
||||
},
|
||||
"transformation_priority": {
|
||||
"type": "string",
|
||||
"description": "Interactive run type means that the query is executed as soon as possible, and these queries count towards concurrent rate limit and daily limit. Read more about interactive run type <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",
|
||||
"default": "interactive",
|
||||
"enum": ["interactive", "batch"],
|
||||
"order": 5,
|
||||
"group": "advanced"
|
||||
},
|
||||
"big_query_client_buffer_size_mb": {
|
||||
"title": "Google BigQuery Client Chunk Size",
|
||||
"description": "Google BigQuery client's chunk (buffer) size (MIN=1, MAX = 15) for each table. The size that will be written by a single RPC. Written data will be buffered and only flushed upon reaching this size or closing the channel. The default 15MB value is used if not set explicitly. Read more <a href=\"https://googleapis.dev/python/bigquery/latest/generated/google.cloud.bigquery.client.Client.html\">here</a>.",
|
||||
"type": "integer",
|
||||
"minimum": 1,
|
||||
"maximum": 15,
|
||||
"default": 15,
|
||||
"examples": ["15"],
|
||||
"order": 6,
|
||||
"group": "advanced"
|
||||
},
|
||||
"raw_data_dataset": {
|
||||
"type": "string",
|
||||
"description": "The dataset to write raw tables into (default: airbyte_internal)",
|
||||
"title": "Raw Table Dataset Name",
|
||||
"order": 7,
|
||||
"group": "advanced"
|
||||
},
|
||||
"disable_type_dedupe": {
|
||||
"type": "boolean",
|
||||
"default": false,
|
||||
"description": "Disable Writing Final Tables. WARNING! The data format in _airbyte_data is likely stable but there are no guarantees that other metadata columns will remain the same in future versions",
|
||||
"title": "Disable Final Tables. (WARNING! Unstable option; Columns in raw table schema might change between versions)",
|
||||
"order": 8,
|
||||
"group": "advanced"
|
||||
}
|
||||
},
|
||||
"groups": [
|
||||
{
|
||||
"id": "connection",
|
||||
"title": "Connection"
|
||||
},
|
||||
{
|
||||
"id": "advanced",
|
||||
"title": "Advanced"
|
||||
}
|
||||
]
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,222 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode
|
||||
import com.google.cloud.bigquery.*
|
||||
import com.google.common.collect.Streams
|
||||
import io.airbyte.cdk.db.bigquery.BigQueryResultSet
|
||||
import io.airbyte.cdk.db.bigquery.BigQuerySourceOperations
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.destination.NamingConventionTransformer
|
||||
import io.airbyte.cdk.integrations.destination.StandardNameTransformer
|
||||
import io.airbyte.cdk.integrations.standardtest.destination.DestinationAcceptanceTest
|
||||
import io.airbyte.cdk.integrations.standardtest.destination.TestingNamespaces.isOlderThan2Days
|
||||
import io.airbyte.cdk.integrations.standardtest.destination.comparator.TestDataComparator
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.commons.string.Strings.addRandomSuffix
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import java.io.IOException
|
||||
import java.nio.file.Path
|
||||
import java.util.*
|
||||
import java.util.function.Function
|
||||
import java.util.stream.Collectors
|
||||
import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.Disabled
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@Disabled
|
||||
abstract class AbstractBigQueryDestinationAcceptanceTest : DestinationAcceptanceTest() {
|
||||
protected var secretsFile: Path? = null
|
||||
protected var bigquery: BigQuery? = null
|
||||
protected var dataset: Dataset? = null
|
||||
|
||||
protected var _config: JsonNode? = null
|
||||
protected val namingResolver: StandardNameTransformer = StandardNameTransformer()
|
||||
|
||||
override val imageName: String
|
||||
get() = "airbyte/destination-bigquery:dev"
|
||||
|
||||
override fun getConfig(): JsonNode {
|
||||
return _config!!
|
||||
}
|
||||
|
||||
override fun getFailCheckConfig(): JsonNode? {
|
||||
(_config as ObjectNode?)!!.put(CONFIG_PROJECT_ID, "fake")
|
||||
return _config
|
||||
}
|
||||
|
||||
override fun implementsNamespaces(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun supportNamespaceTest(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun getTestDataComparator(): TestDataComparator {
|
||||
return BigQueryTestDataComparator()
|
||||
}
|
||||
|
||||
override fun supportBasicDataTypeTest(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun supportArrayDataTypeTest(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun supportObjectDataTypeTest(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun supportIncrementalSchemaChanges(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
override fun getNameTransformer(): Optional<NamingConventionTransformer> {
|
||||
return Optional.of(NAME_TRANSFORMER)
|
||||
}
|
||||
|
||||
override fun assertNamespaceNormalization(
|
||||
testCaseId: String?,
|
||||
expectedNormalizedNamespace: String?,
|
||||
actualNormalizedNamespace: String?
|
||||
) {
|
||||
val message =
|
||||
String.format(
|
||||
"Test case %s failed; if this is expected, please override assertNamespaceNormalization",
|
||||
testCaseId
|
||||
)
|
||||
if (testCaseId == "S3A-1") {
|
||||
/*
|
||||
* See NamespaceTestCaseProvider for how this suffix is generated. <p> expectedNormalizedNamespace
|
||||
* will look something like this: `_99namespace_test_20230824_bicrt`. We want to grab the part after
|
||||
* `_99namespace`.
|
||||
*/
|
||||
val underscoreIndex = expectedNormalizedNamespace!!.indexOf("_", 1)
|
||||
val randomSuffix = expectedNormalizedNamespace.substring(underscoreIndex)
|
||||
/*
|
||||
* bigquery allows originalNamespace starting with a number, and prepending underscore will hide the
|
||||
* dataset, so we don't do it as we do for other destinations
|
||||
*/
|
||||
Assertions.assertEquals("99namespace$randomSuffix", actualNormalizedNamespace, message)
|
||||
} else {
|
||||
Assertions.assertEquals(expectedNormalizedNamespace, actualNormalizedNamespace, message)
|
||||
}
|
||||
}
|
||||
|
||||
override fun getDefaultSchema(config: JsonNode): String? {
|
||||
return getDatasetId(config)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun retrieveRecords(
|
||||
env: TestDestinationEnv?,
|
||||
streamName: String,
|
||||
namespace: String,
|
||||
streamSchema: JsonNode
|
||||
): List<JsonNode> {
|
||||
val streamId =
|
||||
BigQuerySqlGenerator(null, null)
|
||||
.buildStreamId(
|
||||
namespace,
|
||||
streamName,
|
||||
JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE
|
||||
)
|
||||
return retrieveRecordsFromTable(streamId.rawName, streamId.rawNamespace)
|
||||
.stream()
|
||||
.map<String>(
|
||||
Function<JsonNode, String> { node: JsonNode ->
|
||||
node.get(JavaBaseConstants.COLUMN_NAME_DATA).asText()
|
||||
}
|
||||
)
|
||||
.map<JsonNode> { jsonString: String? -> deserialize(jsonString) }
|
||||
.collect(Collectors.toList<JsonNode>())
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
protected fun retrieveRecordsFromTable(tableName: String?, schema: String?): List<JsonNode> {
|
||||
TimeZone.setDefault(TimeZone.getTimeZone("UTC"))
|
||||
|
||||
val queryConfig =
|
||||
QueryJobConfiguration.newBuilder(
|
||||
String.format(
|
||||
"SELECT * FROM `%s`.`%s` order by %s asc;",
|
||||
schema,
|
||||
tableName,
|
||||
JavaBaseConstants.COLUMN_NAME_AB_EXTRACTED_AT
|
||||
)
|
||||
)
|
||||
.setUseLegacySql(false)
|
||||
.setConnectionProperties(
|
||||
listOf<ConnectionProperty>(ConnectionProperty.of("time_zone", "UTC"))
|
||||
)
|
||||
.build()
|
||||
|
||||
val queryResults = executeQuery(bigquery!!, queryConfig).getLeft().getQueryResults()
|
||||
val fields = queryResults.schema!!.fields
|
||||
val sourceOperations = BigQuerySourceOperations()
|
||||
|
||||
return Streams.stream(queryResults.iterateAll())
|
||||
.map { fieldValues: FieldValueList ->
|
||||
sourceOperations.rowToJson(BigQueryResultSet(fieldValues, fields))
|
||||
}
|
||||
.collect(Collectors.toList())
|
||||
}
|
||||
|
||||
@Throws(IOException::class)
|
||||
protected fun setUpBigQuery() {
|
||||
// secrets file should be set by the inhereting class
|
||||
Assertions.assertNotNull(secretsFile)
|
||||
val datasetId = addRandomSuffix("airbyte_tests", "_", 8)
|
||||
val stagingPathSuffix = addRandomSuffix("test_path", "_", 8)
|
||||
val config =
|
||||
BigQueryDestinationTestUtils.createConfig(secretsFile, datasetId, stagingPathSuffix)
|
||||
|
||||
val projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText()
|
||||
this._config = config
|
||||
bigquery = BigQueryDestinationTestUtils.initBigQuery(config, projectId)
|
||||
dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId)
|
||||
}
|
||||
|
||||
protected fun removeOldNamespaces() {
|
||||
var datasetsDeletedCount = 0
|
||||
// todo (cgardens) - hardcoding to testing project to de-risk this running somewhere
|
||||
// unexpected.
|
||||
for (dataset1 in
|
||||
bigquery!!
|
||||
.listDatasets("dataline-integration-testing", BigQuery.DatasetListOption.all())
|
||||
.iterateAll()) {
|
||||
if (isOlderThan2Days(dataset1.datasetId.dataset)) {
|
||||
try {
|
||||
bigquery!!.delete(
|
||||
dataset1.datasetId,
|
||||
BigQuery.DatasetDeleteOption.deleteContents()
|
||||
)
|
||||
datasetsDeletedCount++
|
||||
} catch (e: BigQueryException) {
|
||||
LOGGER.error("Failed to delete old dataset: {}", dataset1.datasetId.dataset, e)
|
||||
}
|
||||
}
|
||||
}
|
||||
LOGGER.info("Deleted {} old datasets.", datasetsDeletedCount)
|
||||
}
|
||||
|
||||
protected fun tearDownBigQuery() {
|
||||
BigQueryDestinationTestUtils.tearDownBigQuery(bigquery, dataset, LOGGER)
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val NAME_TRANSFORMER: NamingConventionTransformer = BigQuerySQLNameTransformer()
|
||||
private val LOGGER: Logger =
|
||||
LoggerFactory.getLogger(AbstractBigQueryDestinationAcceptanceTest::class.java)
|
||||
|
||||
protected const val CONFIG_PROJECT_ID: String = "project_id"
|
||||
}
|
||||
}
|
||||
@@ -1,63 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.load.check.CheckIntegrationTest
|
||||
import io.airbyte.cdk.load.check.CheckTestConfig
|
||||
import io.airbyte.integrations.destination.bigquery.spec.BigquerySpecification
|
||||
import java.util.regex.Pattern
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
class BigQueryCheckTest :
|
||||
CheckIntegrationTest<BigquerySpecification>(
|
||||
successConfigFilenames =
|
||||
listOf(
|
||||
CheckTestConfig(BigQueryDestinationTestUtils.standardInsertConfig),
|
||||
),
|
||||
failConfigFilenamesAndFailureReasons =
|
||||
mapOf(
|
||||
CheckTestConfig(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
"secrets/credentials-badproject.json"
|
||||
),
|
||||
name = "bad project",
|
||||
) to
|
||||
Pattern.compile(
|
||||
"Access Denied: Project fake: User does not have bigquery.datasets.create permission in project fake"
|
||||
),
|
||||
// these tests somehow hang in CI.
|
||||
// CheckTestConfig(
|
||||
// BigQueryDestinationTestUtils.createConfig(
|
||||
// "secrets/credentials-no-edit-public-schema-role.json"
|
||||
// ),
|
||||
// name = "no edit public schema role",
|
||||
// ) to Pattern.compile("Permission bigquery.tables.create denied"),
|
||||
// CheckTestConfig(
|
||||
// BigQueryDestinationTestUtils.createConfig(
|
||||
// "secrets/credentials-standard-no-dataset-creation.json"
|
||||
// ),
|
||||
// name = "no dataset creation",
|
||||
// ) to Pattern.compile("Permission bigquery.tables.create denied"),
|
||||
// somehow this test causes the docker container to emit a malformed log message
|
||||
// (it's truncated).
|
||||
// CheckTestConfig(
|
||||
// BigQueryDestinationTestUtils.createConfig(
|
||||
// "secrets/credentials-1s1t-gcs-bad-copy-permission.json"
|
||||
// ),
|
||||
// name = "gcs bad copy permission",
|
||||
// ) to Pattern.compile("Permission bigquery.tables.updateData denied on table"),
|
||||
),
|
||||
additionalMicronautEnvs = additionalMicronautEnvs,
|
||||
) {
|
||||
@Test
|
||||
override fun testSuccessConfigs() {
|
||||
super.testSuccessConfigs()
|
||||
}
|
||||
|
||||
@Test
|
||||
override fun testFailConfigs() {
|
||||
super.testFailConfigs()
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,732 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.amazonaws.services.s3.AmazonS3
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.google.cloud.bigquery.*
|
||||
import com.google.common.collect.ImmutableMap
|
||||
import com.google.common.collect.Lists
|
||||
import io.airbyte.cdk.integrations.base.AirbyteMessageConsumer
|
||||
import io.airbyte.cdk.integrations.base.Destination
|
||||
import io.airbyte.cdk.integrations.base.DestinationConfig
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.destination.NamingConventionTransformer
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.commons.json.Jsons.jsonNode
|
||||
import io.airbyte.commons.resources.MoreResources.readResource
|
||||
import io.airbyte.commons.string.Strings.addRandomSuffix
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.createPartitionedTableIfNotExists
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.executeQuery
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetLocation
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getGcsJsonNodeConfig
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getOrCreateDataset
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.airbyte.protocol.models.Field
|
||||
import io.airbyte.protocol.models.JsonSchemaType
|
||||
import io.airbyte.protocol.models.v0.*
|
||||
import java.io.IOException
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Path
|
||||
import java.time.Instant
|
||||
import java.util.*
|
||||
import java.util.function.Function
|
||||
import java.util.stream.Collectors
|
||||
import java.util.stream.Stream
|
||||
import java.util.stream.StreamSupport
|
||||
import org.assertj.core.api.Assertions
|
||||
import org.junit.jupiter.api.*
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.Arguments
|
||||
import org.junit.jupiter.params.provider.MethodSource
|
||||
import org.mockito.Mockito
|
||||
import org.mockito.Mockito.mock
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
internal class BigQueryDestinationTest {
|
||||
protected var bigquery: BigQuery? = null
|
||||
protected var dataset: Dataset? = null
|
||||
private var s3Client: AmazonS3? = null
|
||||
|
||||
private fun successTestConfigProviderBase(): Stream<Arguments> {
|
||||
return Stream.of(Arguments.of("config"), Arguments.of("configWithProjectId"))
|
||||
}
|
||||
|
||||
private fun successTestConfigProvider(): Stream<Arguments> {
|
||||
return Stream.concat(
|
||||
successTestConfigProviderBase(),
|
||||
Stream.of(Arguments.of("gcsStagingConfig"))
|
||||
)
|
||||
}
|
||||
|
||||
private fun failCheckTestConfigProvider(): Stream<Arguments> {
|
||||
return Stream.of(
|
||||
Arguments.of(
|
||||
"configWithBadProjectId",
|
||||
"User does not have bigquery.datasets.create permission in project"
|
||||
),
|
||||
Arguments.of(
|
||||
"insufficientRoleConfig",
|
||||
"User does not have bigquery.datasets.create permission"
|
||||
),
|
||||
Arguments.of(
|
||||
"gcsStagingConfigWithBadCopyPermission",
|
||||
"Permission bigquery.tables.updateData denied on table"
|
||||
),
|
||||
)
|
||||
}
|
||||
|
||||
private fun failWriteTestConfigProvider(): Stream<Arguments> {
|
||||
return Stream.of(
|
||||
Arguments.of(
|
||||
"configWithBadProjectId",
|
||||
"User does not have bigquery.datasets.create permission in project"
|
||||
),
|
||||
Arguments.of(
|
||||
"noEditPublicSchemaRoleConfig",
|
||||
"Failed to write to destination schema."
|
||||
), // (or it may not exist)
|
||||
Arguments.of("insufficientRoleConfig", "Permission bigquery.tables.create denied")
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(IOException::class)
|
||||
protected fun initBigQuery(config: JsonNode) {
|
||||
bigquery = BigQueryDestinationTestUtils.initBigQuery(config, projectId)
|
||||
try {
|
||||
dataset = BigQueryDestinationTestUtils.initDataSet(config, bigquery, datasetId)
|
||||
} catch (ex: Exception) {
|
||||
// ignore
|
||||
}
|
||||
}
|
||||
|
||||
@BeforeEach
|
||||
@Throws(IOException::class)
|
||||
fun setup(info: TestInfo) {
|
||||
if (info.displayName == "testSpec()") {
|
||||
return
|
||||
}
|
||||
bigquery = null
|
||||
dataset = null
|
||||
val gcsDestinationConfig: GcsDestinationConfig =
|
||||
GcsDestinationConfig.getGcsDestinationConfig(getGcsJsonNodeConfig(gcsStagingConfig!!))
|
||||
this.s3Client = gcsDestinationConfig.getS3Client()
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
fun tearDown(info: TestInfo) {
|
||||
if (info.displayName == "testSpec()") {
|
||||
return
|
||||
}
|
||||
BigQueryDestinationTestUtils.tearDownBigQuery(bigquery, dataset, LOGGER)
|
||||
BigQueryDestinationTestUtils.tearDownGcs(s3Client, config, LOGGER)
|
||||
}
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
fun testSpec() {
|
||||
val actual = BigQueryDestination().spec()
|
||||
val resourceString = readResource("spec.json")
|
||||
val expected = deserialize(resourceString, ConnectorSpecification::class.java)
|
||||
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expected, actual)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("successTestConfigProvider")
|
||||
@Throws(IOException::class)
|
||||
fun testCheckSuccess(configName: String) {
|
||||
val testConfig = configs!![configName]
|
||||
val actual = BigQueryDestination().check(testConfig!!)
|
||||
val expected =
|
||||
AirbyteConnectionStatus().withStatus(AirbyteConnectionStatus.Status.SUCCEEDED)
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expected, actual)
|
||||
}
|
||||
|
||||
@ParameterizedTest
|
||||
@MethodSource("failCheckTestConfigProvider")
|
||||
fun testCheckFailures(configName: String, error: String?) {
|
||||
// TODO: this should always throw ConfigErrorException
|
||||
val testConfig = configs!![configName]
|
||||
|
||||
val ex =
|
||||
org.junit.jupiter.api.Assertions.assertThrows(Exception::class.java) {
|
||||
BigQueryDestination().check(testConfig!!)
|
||||
}
|
||||
Assertions.assertThat(ex.message).contains(error)
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@ParameterizedTest
|
||||
@MethodSource("successTestConfigProvider")
|
||||
@Throws(Exception::class)
|
||||
fun testWriteSuccess(configName: String) {
|
||||
initBigQuery(config)
|
||||
val testConfig = configs!![configName]
|
||||
val destination = BigQueryDestination()
|
||||
val consumer =
|
||||
destination.getConsumer(testConfig!!, catalog!!) { message: AirbyteMessage? ->
|
||||
Destination.defaultOutputRecordCollector(message)
|
||||
}
|
||||
|
||||
consumer!!.start()
|
||||
consumer.accept(MESSAGE_USERS1)
|
||||
consumer.accept(MESSAGE_TASKS1)
|
||||
consumer.accept(MESSAGE_USERS2)
|
||||
consumer.accept(MESSAGE_TASKS2)
|
||||
consumer.accept(MESSAGE_STATE)
|
||||
consumer.close()
|
||||
|
||||
val usersActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(USERS_STREAM_NAME))
|
||||
val expectedUsersJson: List<JsonNode> =
|
||||
Lists.newArrayList(MESSAGE_USERS1.record.data, MESSAGE_USERS2.record.data)
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expectedUsersJson.size, usersActual.size)
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
expectedUsersJson.containsAll(usersActual) && usersActual.containsAll(expectedUsersJson)
|
||||
)
|
||||
|
||||
val tasksActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(TASKS_STREAM_NAME))
|
||||
val expectedTasksJson: List<JsonNode> =
|
||||
Lists.newArrayList(MESSAGE_TASKS1.record.data, MESSAGE_TASKS2.record.data)
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expectedTasksJson.size, tasksActual.size)
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
expectedTasksJson.containsAll(tasksActual) && tasksActual.containsAll(expectedTasksJson)
|
||||
)
|
||||
|
||||
assertTmpTablesNotPresent(
|
||||
catalog!!
|
||||
.streams
|
||||
.stream()
|
||||
.map { obj: ConfiguredAirbyteStream -> obj.stream }
|
||||
.map { obj: AirbyteStream -> obj.name }
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
fun testCreateTableSuccessWhenTableAlreadyExists() {
|
||||
initBigQuery(config)
|
||||
|
||||
// Test schema where we will try to re-create existing table
|
||||
val tmpTestSchemaName = "test_create_table_when_exists_schema"
|
||||
|
||||
val schema =
|
||||
Schema.of(
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_ID,
|
||||
StandardSQLTypeName.STRING
|
||||
),
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_EMITTED_AT,
|
||||
StandardSQLTypeName.TIMESTAMP
|
||||
),
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_DATA,
|
||||
StandardSQLTypeName.STRING
|
||||
)
|
||||
)
|
||||
|
||||
val tableId = TableId.of(tmpTestSchemaName, "test_already_existing_table")
|
||||
|
||||
getOrCreateDataset(bigquery!!, tmpTestSchemaName, getDatasetLocation(config!!))
|
||||
|
||||
org.junit.jupiter.api.Assertions.assertDoesNotThrow {
|
||||
// Create table
|
||||
createPartitionedTableIfNotExists(bigquery!!, tableId, schema)
|
||||
|
||||
// Try to create it one more time. Shouldn't throw exception
|
||||
createPartitionedTableIfNotExists(bigquery!!, tableId, schema)
|
||||
}
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@ParameterizedTest
|
||||
@MethodSource("failWriteTestConfigProvider")
|
||||
@Throws(Exception::class)
|
||||
fun testWriteFailure(configName: String, error: String?) {
|
||||
initBigQuery(config)
|
||||
val testConfig = configs!![configName]
|
||||
val ex =
|
||||
org.junit.jupiter.api.Assertions.assertThrows<Exception>(Exception::class.java) {
|
||||
val consumer =
|
||||
Mockito.spy<AirbyteMessageConsumer?>(
|
||||
BigQueryDestination().getConsumer(testConfig!!, catalog!!) {
|
||||
message: AirbyteMessage? ->
|
||||
Destination.defaultOutputRecordCollector(message)
|
||||
}
|
||||
)
|
||||
consumer!!.start()
|
||||
}
|
||||
Assertions.assertThat(ex.message).contains(error)
|
||||
|
||||
val tableNames =
|
||||
catalog!!
|
||||
.streams
|
||||
.stream()
|
||||
.map { obj: ConfiguredAirbyteStream -> obj.stream }
|
||||
.map { obj: AirbyteStream -> obj.name }
|
||||
.toList()
|
||||
assertTmpTablesNotPresent(
|
||||
catalog!!
|
||||
.streams
|
||||
.stream()
|
||||
.map { obj: ConfiguredAirbyteStream -> obj.stream }
|
||||
.map { obj: AirbyteStream -> obj.name }
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
// assert that no tables were created.
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
fetchNamesOfTablesInDb().stream().noneMatch { tableName: String ->
|
||||
tableNames.stream().anyMatch { prefix: String -> tableName.startsWith(prefix) }
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
private fun fetchNamesOfTablesInDb(): Set<String> {
|
||||
if (dataset == null || bigquery == null) {
|
||||
return emptySet()
|
||||
}
|
||||
val queryConfig =
|
||||
QueryJobConfiguration.newBuilder(
|
||||
String.format(
|
||||
"SELECT * FROM `%s.INFORMATION_SCHEMA.TABLES`;",
|
||||
dataset!!.datasetId.dataset
|
||||
)
|
||||
)
|
||||
.setUseLegacySql(false)
|
||||
.build()
|
||||
|
||||
if (!dataset!!.exists()) {
|
||||
return emptySet()
|
||||
}
|
||||
return StreamSupport.stream(
|
||||
executeQuery(bigquery!!, queryConfig)
|
||||
.getLeft()
|
||||
.getQueryResults()
|
||||
.iterateAll()
|
||||
.spliterator(),
|
||||
false
|
||||
)
|
||||
.map { v: FieldValueList -> v["TABLE_NAME"].stringValue }
|
||||
.collect(Collectors.toSet())
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
private fun assertTmpTablesNotPresent(tableNames: List<String>) {
|
||||
val tmpTableNamePrefixes =
|
||||
tableNames.stream().map { name: String -> name + "_" }.collect(Collectors.toSet())
|
||||
val finalTableNames =
|
||||
tableNames.stream().map { name: String -> name + "_raw" }.collect(Collectors.toSet())
|
||||
// search for table names that have the tmp table prefix but are not raw tables.
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
fetchNamesOfTablesInDb()
|
||||
.stream()
|
||||
.filter { tableName: String -> !finalTableNames.contains(tableName) }
|
||||
.noneMatch { tableName: String ->
|
||||
tmpTableNamePrefixes.stream().anyMatch { prefix: String ->
|
||||
tableName.startsWith(prefix)
|
||||
}
|
||||
}
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
private fun retrieveRecords(tableName: String): List<JsonNode> {
|
||||
val queryConfig =
|
||||
QueryJobConfiguration.newBuilder(
|
||||
String.format(
|
||||
"SELECT * FROM `%s.%s`;",
|
||||
dataset!!.datasetId.dataset,
|
||||
tableName.lowercase(Locale.getDefault())
|
||||
)
|
||||
)
|
||||
.setUseLegacySql(false)
|
||||
.build()
|
||||
|
||||
executeQuery(bigquery!!, queryConfig)
|
||||
|
||||
return StreamSupport.stream<FieldValueList>(
|
||||
executeQuery(bigquery!!, queryConfig)
|
||||
.getLeft()
|
||||
.getQueryResults()
|
||||
.iterateAll()
|
||||
.spliterator(),
|
||||
false
|
||||
)
|
||||
.map<String>(
|
||||
Function<FieldValueList, String> { v: FieldValueList ->
|
||||
v.get(JavaBaseConstants.COLUMN_NAME_DATA).getStringValue()
|
||||
}
|
||||
)
|
||||
.map<JsonNode> { jsonString: String? -> deserialize(jsonString) }
|
||||
.collect(Collectors.toList<JsonNode>())
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@ParameterizedTest
|
||||
@MethodSource("successTestConfigProviderBase")
|
||||
@Throws(Exception::class)
|
||||
fun testWritePartitionOverUnpartitioned(configName: String) {
|
||||
val testConfig = configs!![configName]
|
||||
initBigQuery(config)
|
||||
val streamId =
|
||||
BigQuerySqlGenerator(projectId, null)
|
||||
.buildStreamId(
|
||||
datasetId!!,
|
||||
USERS_STREAM_NAME,
|
||||
JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE
|
||||
)
|
||||
val dataset =
|
||||
BigQueryDestinationTestUtils.initDataSet(config, bigquery, streamId.rawNamespace)
|
||||
createUnpartitionedTable(bigquery!!, dataset, streamId.rawName)
|
||||
org.junit.jupiter.api.Assertions.assertFalse(
|
||||
isTablePartitioned(bigquery!!, dataset, streamId.rawName)
|
||||
)
|
||||
val destination = BigQueryDestination()
|
||||
val consumer =
|
||||
destination.getConsumer(testConfig!!, catalog!!) { message: AirbyteMessage? ->
|
||||
Destination.defaultOutputRecordCollector(message)
|
||||
}
|
||||
|
||||
consumer!!.start()
|
||||
consumer.accept(MESSAGE_USERS1)
|
||||
consumer.accept(MESSAGE_TASKS1)
|
||||
consumer.accept(MESSAGE_USERS2)
|
||||
consumer.accept(MESSAGE_TASKS2)
|
||||
consumer.accept(MESSAGE_STATE)
|
||||
consumer.close()
|
||||
|
||||
val usersActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(USERS_STREAM_NAME))
|
||||
val expectedUsersJson: List<JsonNode> =
|
||||
Lists.newArrayList(MESSAGE_USERS1.record.data, MESSAGE_USERS2.record.data)
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expectedUsersJson.size, usersActual.size)
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
expectedUsersJson.containsAll(usersActual) && usersActual.containsAll(expectedUsersJson)
|
||||
)
|
||||
|
||||
val tasksActual = retrieveRecords(NAMING_RESOLVER.getRawTableName(TASKS_STREAM_NAME))
|
||||
val expectedTasksJson: List<JsonNode> =
|
||||
Lists.newArrayList(MESSAGE_TASKS1.record.data, MESSAGE_TASKS2.record.data)
|
||||
org.junit.jupiter.api.Assertions.assertEquals(expectedTasksJson.size, tasksActual.size)
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
expectedTasksJson.containsAll(tasksActual) && tasksActual.containsAll(expectedTasksJson)
|
||||
)
|
||||
|
||||
assertTmpTablesNotPresent(
|
||||
catalog!!
|
||||
.streams
|
||||
.stream()
|
||||
.map { obj: ConfiguredAirbyteStream -> obj.stream }
|
||||
.map { obj: AirbyteStream -> obj.name }
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
org.junit.jupiter.api.Assertions.assertTrue(
|
||||
isTablePartitioned(bigquery!!, dataset, streamId.rawName)
|
||||
)
|
||||
}
|
||||
|
||||
private fun createUnpartitionedTable(
|
||||
bigquery: BigQuery?,
|
||||
dataset: Dataset?,
|
||||
tableName: String
|
||||
) {
|
||||
val tableId = TableId.of(dataset!!.datasetId.dataset, tableName)
|
||||
bigquery!!.delete(tableId)
|
||||
val schema =
|
||||
Schema.of(
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_ID,
|
||||
StandardSQLTypeName.STRING
|
||||
),
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_EMITTED_AT,
|
||||
StandardSQLTypeName.TIMESTAMP
|
||||
),
|
||||
com.google.cloud.bigquery.Field.of(
|
||||
JavaBaseConstants.COLUMN_NAME_DATA,
|
||||
StandardSQLTypeName.STRING
|
||||
)
|
||||
)
|
||||
val tableDefinition = StandardTableDefinition.newBuilder().setSchema(schema).build()
|
||||
val tableInfo = TableInfo.newBuilder(tableId, tableDefinition).build()
|
||||
bigquery.create(tableInfo)
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
private fun isTablePartitioned(
|
||||
bigquery: BigQuery?,
|
||||
dataset: Dataset?,
|
||||
tableName: String
|
||||
): Boolean {
|
||||
val queryConfig =
|
||||
QueryJobConfiguration.newBuilder(
|
||||
String.format(
|
||||
"SELECT max(is_partitioning_column) as is_partitioned FROM `%s.%s.INFORMATION_SCHEMA.COLUMNS` WHERE TABLE_NAME = '%s';",
|
||||
bigquery!!.options.projectId,
|
||||
dataset!!.datasetId.dataset,
|
||||
tableName
|
||||
)
|
||||
)
|
||||
.setUseLegacySql(false)
|
||||
.build()
|
||||
val result = executeQuery(bigquery, queryConfig)
|
||||
for (row in result.getLeft().getQueryResults().values) {
|
||||
return !row["is_partitioned"].isNull && row["is_partitioned"].stringValue == "YES"
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
companion object {
|
||||
protected val CREDENTIALS_STANDARD_INSERT_PATH: Path =
|
||||
Path.of("secrets/credentials-standard.json")
|
||||
protected val CREDENTIALS_BAD_PROJECT_PATH: Path =
|
||||
Path.of("secrets/credentials-badproject.json")
|
||||
protected val CREDENTIALS_NO_DATASET_CREATION_PATH: Path =
|
||||
Path.of("secrets/credentials-standard-no-dataset-creation.json")
|
||||
protected val CREDENTIALS_NON_BILLABLE_PROJECT_PATH: Path =
|
||||
Path.of("secrets/credentials-standard-non-billable-project.json")
|
||||
protected val CREDENTIALS_NO_EDIT_PUBLIC_SCHEMA_ROLE_PATH: Path =
|
||||
Path.of("secrets/credentials-no-edit-public-schema-role.json")
|
||||
protected val CREDENTIALS_WITH_GCS_STAGING_PATH: Path =
|
||||
Path.of("secrets/credentials-gcs-staging.json")
|
||||
protected val CREDENTIALS_WITH_GCS_BAD_COPY_PERMISSION_PATH: Path =
|
||||
Path.of("secrets/credentials-1s1t-gcs-bad-copy-permission.json")
|
||||
|
||||
protected val ALL_PATHS: Array<Path> =
|
||||
arrayOf(
|
||||
CREDENTIALS_STANDARD_INSERT_PATH,
|
||||
CREDENTIALS_BAD_PROJECT_PATH,
|
||||
CREDENTIALS_NO_DATASET_CREATION_PATH,
|
||||
CREDENTIALS_NO_EDIT_PUBLIC_SCHEMA_ROLE_PATH,
|
||||
CREDENTIALS_NON_BILLABLE_PROJECT_PATH,
|
||||
CREDENTIALS_WITH_GCS_STAGING_PATH,
|
||||
CREDENTIALS_WITH_GCS_BAD_COPY_PERMISSION_PATH
|
||||
)
|
||||
|
||||
private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDestinationTest::class.java)
|
||||
private const val DATASET_NAME_PREFIX = "bq_dest_integration_test"
|
||||
|
||||
private val NOW: Instant = Instant.now()
|
||||
protected const val USERS_STREAM_NAME: String = "users"
|
||||
protected const val TASKS_STREAM_NAME: String = "tasks"
|
||||
protected val MESSAGE_USERS1: AirbyteMessage =
|
||||
AirbyteMessage()
|
||||
.withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(
|
||||
AirbyteRecordMessage()
|
||||
.withStream(USERS_STREAM_NAME)
|
||||
.withData(
|
||||
jsonNode(
|
||||
ImmutableMap.builder<Any, Any>()
|
||||
.put("name", "john")
|
||||
.put("id", "10")
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.withEmittedAt(NOW.toEpochMilli())
|
||||
)
|
||||
protected val MESSAGE_USERS2: AirbyteMessage =
|
||||
AirbyteMessage()
|
||||
.withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(
|
||||
AirbyteRecordMessage()
|
||||
.withStream(USERS_STREAM_NAME)
|
||||
.withData(
|
||||
jsonNode(
|
||||
ImmutableMap.builder<Any, Any>()
|
||||
.put("name", "susan")
|
||||
.put("id", "30")
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.withEmittedAt(NOW.toEpochMilli())
|
||||
)
|
||||
protected val MESSAGE_TASKS1: AirbyteMessage =
|
||||
AirbyteMessage()
|
||||
.withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(
|
||||
AirbyteRecordMessage()
|
||||
.withStream(TASKS_STREAM_NAME)
|
||||
.withData(
|
||||
jsonNode(
|
||||
ImmutableMap.builder<Any, Any>()
|
||||
.put("goal", "announce the game.")
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.withEmittedAt(NOW.toEpochMilli())
|
||||
)
|
||||
protected val MESSAGE_TASKS2: AirbyteMessage =
|
||||
AirbyteMessage()
|
||||
.withType(AirbyteMessage.Type.RECORD)
|
||||
.withRecord(
|
||||
AirbyteRecordMessage()
|
||||
.withStream(TASKS_STREAM_NAME)
|
||||
.withData(
|
||||
jsonNode(
|
||||
ImmutableMap.builder<Any, Any>()
|
||||
.put("goal", "ship some code.")
|
||||
.build()
|
||||
)
|
||||
)
|
||||
.withEmittedAt(NOW.toEpochMilli())
|
||||
)
|
||||
protected val MESSAGE_STATE: AirbyteMessage =
|
||||
AirbyteMessage()
|
||||
.withType(AirbyteMessage.Type.STATE)
|
||||
.withState(
|
||||
AirbyteStateMessage()
|
||||
.withData(
|
||||
jsonNode(
|
||||
ImmutableMap.builder<Any, Any>().put("checkpoint", "now!").build()
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
private val NAMING_RESOLVER: NamingConventionTransformer = BigQuerySQLNameTransformer()
|
||||
|
||||
protected var projectId: String? = null
|
||||
protected var datasetId: String? = null
|
||||
protected var config: JsonNode = mock()
|
||||
protected var configWithProjectId: JsonNode? = null
|
||||
protected var configWithBadProjectId: JsonNode? = null
|
||||
protected var insufficientRoleConfig: JsonNode? = null
|
||||
protected var noEditPublicSchemaRoleConfig: JsonNode? = null
|
||||
protected var nonBillableConfig: JsonNode? = null
|
||||
protected var gcsStagingConfig: JsonNode? =
|
||||
null // default BigQuery config. Also used for setup/teardown
|
||||
protected var gcsStagingConfigWithBadCopyPermission: JsonNode? = null
|
||||
|
||||
protected var configs: Map<String, JsonNode>? = null
|
||||
protected var catalog: ConfiguredAirbyteCatalog? = null
|
||||
|
||||
@BeforeAll
|
||||
@Throws(IOException::class)
|
||||
@JvmStatic
|
||||
fun beforeAll() {
|
||||
for (path in ALL_PATHS) {
|
||||
check(Files.exists(path)) {
|
||||
String.format(
|
||||
"Must provide path to a big query credentials file. Please add file with credentials to %s",
|
||||
path.toAbsolutePath()
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
datasetId = addRandomSuffix(DATASET_NAME_PREFIX, "_", 8)
|
||||
val stagingPath = addRandomSuffix("test_path", "_", 8)
|
||||
// Set up config objects for test scenarios
|
||||
// config - basic config for standard inserts that should succeed check and write tests
|
||||
// this config is also used for housekeeping (checking records, and cleaning up)
|
||||
val config: JsonNode =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_STANDARD_INSERT_PATH,
|
||||
datasetId,
|
||||
stagingPath
|
||||
)
|
||||
|
||||
DestinationConfig.initialize(config)
|
||||
|
||||
// all successful configs use the same project ID
|
||||
projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText()
|
||||
this.config = config
|
||||
|
||||
// configWithProjectId - config that uses project:dataset notation for rawNamespace
|
||||
val dataSetWithProjectId = String.format("%s:%s", projectId, datasetId)
|
||||
configWithProjectId =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_STANDARD_INSERT_PATH,
|
||||
dataSetWithProjectId,
|
||||
stagingPath
|
||||
)
|
||||
|
||||
// configWithBadProjectId - config that uses "fake" project ID and should fail
|
||||
val dataSetWithBadProjectId = String.format("%s:%s", "fake", datasetId)
|
||||
configWithBadProjectId =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_BAD_PROJECT_PATH,
|
||||
dataSetWithBadProjectId,
|
||||
stagingPath
|
||||
)
|
||||
|
||||
// config that has insufficient privileges
|
||||
insufficientRoleConfig =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_NO_DATASET_CREATION_PATH,
|
||||
datasetId,
|
||||
stagingPath
|
||||
)
|
||||
// config that tries to write to a project with disabled billing (free tier)
|
||||
nonBillableConfig =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_NON_BILLABLE_PROJECT_PATH,
|
||||
"testnobilling",
|
||||
stagingPath
|
||||
)
|
||||
// config that has no privileges to edit anything in Public schema
|
||||
noEditPublicSchemaRoleConfig =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_NO_EDIT_PUBLIC_SCHEMA_ROLE_PATH,
|
||||
"public",
|
||||
stagingPath
|
||||
)
|
||||
// config with GCS staging
|
||||
gcsStagingConfig =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_WITH_GCS_STAGING_PATH,
|
||||
datasetId,
|
||||
stagingPath
|
||||
)
|
||||
|
||||
gcsStagingConfigWithBadCopyPermission =
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
CREDENTIALS_WITH_GCS_BAD_COPY_PERMISSION_PATH,
|
||||
datasetId,
|
||||
stagingPath
|
||||
)
|
||||
|
||||
MESSAGE_USERS1.record.namespace = datasetId
|
||||
MESSAGE_USERS2.record.namespace = datasetId
|
||||
MESSAGE_TASKS1.record.namespace = datasetId
|
||||
MESSAGE_TASKS2.record.namespace = datasetId
|
||||
|
||||
catalog =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
Lists.newArrayList(
|
||||
CatalogHelpers.createConfiguredAirbyteStream(
|
||||
USERS_STREAM_NAME,
|
||||
datasetId,
|
||||
Field.of("name", JsonSchemaType.STRING),
|
||||
Field.of("id", JsonSchemaType.STRING)
|
||||
)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND),
|
||||
CatalogHelpers.createConfiguredAirbyteStream(
|
||||
TASKS_STREAM_NAME,
|
||||
datasetId,
|
||||
Field.of("goal", JsonSchemaType.STRING)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
configs =
|
||||
mapOf(
|
||||
"config" to config,
|
||||
"configWithProjectId" to configWithProjectId!!,
|
||||
"configWithBadProjectId" to configWithBadProjectId!!,
|
||||
"insufficientRoleConfig" to insufficientRoleConfig!!,
|
||||
"noEditPublicSchemaRoleConfig" to noEditPublicSchemaRoleConfig!!,
|
||||
"nonBillableConfig" to nonBillableConfig!!,
|
||||
"gcsStagingConfig" to gcsStagingConfig!!,
|
||||
"gcsStagingConfigWithBadCopyPermission" to
|
||||
gcsStagingConfigWithBadCopyPermission!!,
|
||||
)
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -3,31 +3,26 @@
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.amazonaws.services.s3.AmazonS3
|
||||
import com.amazonaws.services.s3.model.DeleteObjectsRequest
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode
|
||||
import io.airbyte.cdk.load.util.Jsons
|
||||
import io.airbyte.cdk.load.util.deserializeToNode
|
||||
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 com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryOptions
|
||||
import com.google.cloud.bigquery.Dataset
|
||||
import com.google.cloud.bigquery.DatasetInfo
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getServiceAccountCredentials
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getLoadingMethod
|
||||
import java.io.IOException
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Path
|
||||
import java.util.*
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
object BigQueryDestinationTestUtils {
|
||||
const val RAW_DATASET_OVERRIDE = "overridden_raw_dataset"
|
||||
const val STANDARD_INSERT_CONFIG = "secrets/credentials-1s1t-standard.json"
|
||||
const val GCS_STAGING_CONFIG = "secrets/credentials-1s1t-gcs.json"
|
||||
|
||||
val standardInsertRawOverrideConfig =
|
||||
createConfig(
|
||||
configFile = STANDARD_INSERT_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
)
|
||||
val standardInsertConfig =
|
||||
createConfig(
|
||||
configFile = STANDARD_INSERT_CONFIG,
|
||||
)
|
||||
private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryDestinationTestUtils::class.java)
|
||||
|
||||
/**
|
||||
* Parse the config file and replace dataset with rawNamespace and stagingPath randomly
|
||||
@@ -35,25 +30,127 @@ object BigQueryDestinationTestUtils {
|
||||
*
|
||||
* @param configFile Path to the config file
|
||||
* @param datasetId Dataset id to use in the test. Should be randomized per test case.
|
||||
* @param stagingPath Staging GCS path to use in the test, or null if the test is running in
|
||||
* standard inserts mode. Should be randomized per test case.
|
||||
*/
|
||||
@Throws(IOException::class)
|
||||
fun createConfig(
|
||||
configFile: String,
|
||||
rawDatasetId: String? = null,
|
||||
disableTypingDeduping: Boolean? = null,
|
||||
): String {
|
||||
val tmpConfigAsString = Files.readString(Path.of(configFile))
|
||||
val config = Jsons.readTree(tmpConfigAsString) as ObjectNode
|
||||
config.put(BigQueryConsts.CONFIG_DATASET_ID, DEFAULT_NAMESPACE_PLACEHOLDER)
|
||||
rawDatasetId?.let { config.put(BigQueryConsts.RAW_DATA_DATASET, rawDatasetId) }
|
||||
disableTypingDeduping?.let {
|
||||
config.put(BigQueryConsts.DISABLE_TYPE_DEDUPE, disableTypingDeduping)
|
||||
fun createConfig(configFile: Path?, datasetId: String?, stagingPath: String?): ObjectNode {
|
||||
LOGGER.info("Setting default dataset to {}", datasetId)
|
||||
val tmpConfigAsString = Files.readString(configFile)
|
||||
val config = deserialize(tmpConfigAsString) as ObjectNode
|
||||
config.put(BigQueryConsts.CONFIG_DATASET_ID, datasetId)
|
||||
|
||||
// This is sort of a hack. Ideally tests shouldn't interfere with each other even when using
|
||||
// the
|
||||
// same staging path.
|
||||
// Most likely there's a real bug in the connector - but we should investigate that and
|
||||
// write a real
|
||||
// test case,
|
||||
// rather than relying on tests randomly failing to indicate that bug.
|
||||
// See https://github.com/airbytehq/airbyte/issues/28372.
|
||||
if (stagingPath != null && getLoadingMethod(config) == UploadingMethod.GCS) {
|
||||
val loadingMethodNode = config[BigQueryConsts.LOADING_METHOD] as ObjectNode
|
||||
loadingMethodNode.put(BigQueryConsts.GCS_BUCKET_PATH, stagingPath)
|
||||
}
|
||||
return config.serializeToString()
|
||||
return config
|
||||
}
|
||||
|
||||
fun parseConfig(config: String): BigqueryConfiguration {
|
||||
val spec = Jsons.treeToValue(config.deserializeToNode(), BigquerySpecification::class.java)
|
||||
return BigqueryConfigurationFactory().make(spec)
|
||||
/**
|
||||
* Get a handle for the BigQuery dataset instance used by the test. This dataset instance will
|
||||
* be used to verify results of test operations and for cleaning up after the test runs
|
||||
*
|
||||
* @param config
|
||||
* @param bigquery
|
||||
* @param datasetId
|
||||
* @return
|
||||
*/
|
||||
fun initDataSet(config: JsonNode, bigquery: BigQuery?, datasetId: String?): Dataset? {
|
||||
val datasetInfo =
|
||||
DatasetInfo.newBuilder(datasetId)
|
||||
.setLocation(config[BigQueryConsts.CONFIG_DATASET_LOCATION].asText())
|
||||
.build()
|
||||
try {
|
||||
return bigquery!!.create(datasetInfo)
|
||||
} catch (ex: Exception) {
|
||||
if (ex.message!!.indexOf("Already Exists") > -1) {
|
||||
return bigquery!!.getDataset(datasetId)
|
||||
}
|
||||
}
|
||||
return null
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialized bigQuery instance that will be used for verifying results of test operations and
|
||||
* for cleaning up BigQuery dataset after the test
|
||||
*
|
||||
* @param config
|
||||
* @param projectId
|
||||
* @return
|
||||
* @throws IOException
|
||||
*/
|
||||
@Throws(IOException::class)
|
||||
fun initBigQuery(config: JsonNode, projectId: String?): BigQuery {
|
||||
val credentials = getServiceAccountCredentials(config)
|
||||
return BigQueryOptions.newBuilder()
|
||||
.setProjectId(projectId)
|
||||
.setCredentials(credentials)
|
||||
.build()
|
||||
.service
|
||||
}
|
||||
|
||||
/**
|
||||
* Deletes bigquery data set created during the test
|
||||
*
|
||||
* @param bigquery
|
||||
* @param dataset
|
||||
* @param LOGGER
|
||||
*/
|
||||
fun tearDownBigQuery(bigquery: BigQuery?, dataset: Dataset?, LOGGER: Logger) {
|
||||
// allows deletion of a dataset that has contents
|
||||
val option = BigQuery.DatasetDeleteOption.deleteContents()
|
||||
if (bigquery == null || dataset == null) {
|
||||
return
|
||||
}
|
||||
try {
|
||||
val success = bigquery.delete(dataset.datasetId, option)
|
||||
if (success) {
|
||||
LOGGER.info("BQ Dataset $dataset deleted...")
|
||||
} else {
|
||||
LOGGER.info("BQ Dataset cleanup for $dataset failed!")
|
||||
}
|
||||
} catch (ex: Exception) {
|
||||
LOGGER.error("Failed to remove BigQuery resources after the test", ex)
|
||||
}
|
||||
}
|
||||
|
||||
/** Remove all the GCS output from the tests. */
|
||||
fun tearDownGcs(s3Client: AmazonS3?, config: JsonNode?, LOGGER: Logger) {
|
||||
if (s3Client == null) {
|
||||
return
|
||||
}
|
||||
if (getLoadingMethod(config!!) != UploadingMethod.GCS) {
|
||||
return
|
||||
}
|
||||
val properties = config[BigQueryConsts.LOADING_METHOD]
|
||||
val gcsBucketName = properties[BigQueryConsts.GCS_BUCKET_NAME].asText()
|
||||
val gcs_bucket_path = properties[BigQueryConsts.GCS_BUCKET_PATH].asText()
|
||||
try {
|
||||
val keysToDelete: MutableList<DeleteObjectsRequest.KeyVersion> = LinkedList()
|
||||
val objects = s3Client.listObjects(gcsBucketName, gcs_bucket_path).objectSummaries
|
||||
for (`object` in objects) {
|
||||
keysToDelete.add(DeleteObjectsRequest.KeyVersion(`object`.key))
|
||||
}
|
||||
|
||||
if (keysToDelete.size > 0) {
|
||||
LOGGER.info("Tearing down test bucket path: {}/{}", gcsBucketName, gcs_bucket_path)
|
||||
// Google Cloud Storage doesn't accept request to delete multiple objects
|
||||
for (keyToDelete in keysToDelete) {
|
||||
s3Client.deleteObject(gcsBucketName, keyToDelete.key)
|
||||
}
|
||||
LOGGER.info("Deleted {} file(s).", keysToDelete.size)
|
||||
}
|
||||
} catch (ex: Exception) {
|
||||
LOGGER.error("Failed to remove GCS resources after the test", ex)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@@ -0,0 +1,69 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.amazonaws.services.s3.AmazonS3
|
||||
import io.airbyte.cdk.integrations.base.DestinationConfig
|
||||
import io.airbyte.cdk.integrations.destination.gcs.GcsDestinationConfig
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getGcsJsonNodeConfig
|
||||
import java.nio.file.Path
|
||||
import org.junit.jupiter.api.Disabled
|
||||
import org.junit.jupiter.api.TestInstance
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@Disabled
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
class BigQueryGcsDestinationAcceptanceTest : AbstractBigQueryDestinationAcceptanceTest() {
|
||||
private var s3Client: AmazonS3? = null
|
||||
|
||||
/**
|
||||
* Sets up secretsFile path as well as BigQuery and GCS instances for verification and cleanup
|
||||
* This function will be called before EACH test.
|
||||
*
|
||||
* @param testEnv
|
||||
* - information about the test environment.
|
||||
* @param TEST_SCHEMAS
|
||||
* @throws Exception
|
||||
* - can throw any exception, test framework will handle.
|
||||
* @see DestinationAcceptanceTest.setUpInternal
|
||||
*/
|
||||
@Throws(Exception::class)
|
||||
override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet<String>) {
|
||||
// use secrets file with GCS staging config
|
||||
secretsFile = Path.of("secrets/credentials-gcs-staging.json")
|
||||
setUpBigQuery()
|
||||
removeOldNamespaces()
|
||||
|
||||
DestinationConfig.initialize(_config)
|
||||
|
||||
// the setup steps below are specific to GCS staging use case
|
||||
val gcsDestinationConfig: GcsDestinationConfig =
|
||||
GcsDestinationConfig.getGcsDestinationConfig(getGcsJsonNodeConfig(_config!!))
|
||||
this.s3Client = gcsDestinationConfig.getS3Client()
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes data from bigquery and GCS This function will be called after EACH test
|
||||
*
|
||||
* @param testEnv
|
||||
* - information about the test environment.
|
||||
* @throws Exception
|
||||
* - can throw any exception, test framework will handle.
|
||||
* @see DestinationAcceptanceTest.tearDownInternal
|
||||
*/
|
||||
override fun tearDown(testEnv: TestDestinationEnv) {
|
||||
tearDownBigQuery()
|
||||
tearDownGcs()
|
||||
}
|
||||
|
||||
protected fun tearDownGcs() {
|
||||
BigQueryDestinationTestUtils.tearDownGcs(s3Client, _config, LOGGER)
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val LOGGER: Logger =
|
||||
LoggerFactory.getLogger(BigQueryGcsDestinationAcceptanceTest::class.java)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,50 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import java.nio.file.Path
|
||||
import org.junit.jupiter.api.Disabled
|
||||
import org.junit.jupiter.api.TestInstance
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@Disabled
|
||||
@TestInstance(TestInstance.Lifecycle.PER_CLASS)
|
||||
class BigQueryStandardDestinationAcceptanceTest : AbstractBigQueryDestinationAcceptanceTest() {
|
||||
/**
|
||||
* Sets up secretsFile path and BigQuery instance for verification and cleanup This function
|
||||
* will be called before EACH test.
|
||||
*
|
||||
* @param testEnv
|
||||
* - information about the test environment.
|
||||
* @param TEST_SCHEMAS
|
||||
* @throws Exception
|
||||
* - can throw any exception, test framework will handle.
|
||||
* @see DestinationAcceptanceTest.setUpInternal
|
||||
*/
|
||||
@Throws(Exception::class)
|
||||
override fun setup(testEnv: TestDestinationEnv, TEST_SCHEMAS: HashSet<String>) {
|
||||
secretsFile = Path.of("secrets/credentials-standard.json")
|
||||
setUpBigQuery()
|
||||
removeOldNamespaces()
|
||||
}
|
||||
|
||||
/**
|
||||
* Removes data from bigquery This function will be called after EACH test
|
||||
*
|
||||
* @param testEnv
|
||||
* - information about the test environment.
|
||||
* @throws Exception
|
||||
* - can throw any exception, test framework will handle.
|
||||
* @see DestinationAcceptanceTest.tearDownInternal
|
||||
*/
|
||||
override fun tearDown(testEnv: TestDestinationEnv) {
|
||||
tearDownBigQuery()
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val LOGGER: Logger =
|
||||
LoggerFactory.getLogger(BigQueryStandardDestinationAcceptanceTest::class.java)
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,118 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.cdk.integrations.destination.StandardNameTransformer
|
||||
import io.airbyte.cdk.integrations.standardtest.destination.comparator.AdvancedTestDataComparator
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import java.time.LocalDate
|
||||
import java.time.LocalDateTime
|
||||
import java.time.ZoneOffset
|
||||
import java.time.ZonedDateTime
|
||||
import java.time.format.DateTimeFormatter
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
class BigQueryTestDataComparator : AdvancedTestDataComparator() {
|
||||
private val namingResolver = StandardNameTransformer()
|
||||
|
||||
@SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE")
|
||||
override fun resolveIdentifier(identifier: String?): List<String?> {
|
||||
val result: MutableList<String?> = ArrayList()
|
||||
result.add(identifier)
|
||||
result.add(namingResolver.getIdentifier(identifier!!))
|
||||
return result
|
||||
}
|
||||
|
||||
private fun parseDate(dateValue: String?): LocalDate? {
|
||||
if (dateValue != null) {
|
||||
val format =
|
||||
(if (dateValue.matches(".+Z".toRegex())) BIGQUERY_DATETIME_FORMAT
|
||||
else AIRBYTE_DATE_FORMAT)
|
||||
return LocalDate.parse(dateValue, DateTimeFormatter.ofPattern(format))
|
||||
} else {
|
||||
return null
|
||||
}
|
||||
}
|
||||
|
||||
private fun parseDateTime(dateTimeValue: String?): LocalDateTime? {
|
||||
if (dateTimeValue != null) {
|
||||
val format =
|
||||
(if (dateTimeValue.matches(".+Z".toRegex())) BIGQUERY_DATETIME_FORMAT
|
||||
else AIRBYTE_DATETIME_FORMAT)
|
||||
return LocalDateTime.parse(dateTimeValue, DateTimeFormatter.ofPattern(format))
|
||||
} else {
|
||||
return null
|
||||
}
|
||||
}
|
||||
|
||||
override fun parseDestinationDateWithTz(destinationValue: String): ZonedDateTime {
|
||||
return if (destinationValue.matches(".+Z".toRegex())) {
|
||||
ZonedDateTime.of(
|
||||
LocalDateTime.parse(
|
||||
destinationValue,
|
||||
DateTimeFormatter.ofPattern(BIGQUERY_DATETIME_FORMAT)
|
||||
),
|
||||
ZoneOffset.UTC
|
||||
)
|
||||
} else {
|
||||
ZonedDateTime.parse(destinationValue, airbyteDateTimeWithTzFormatter)
|
||||
.withZoneSameInstant(ZoneOffset.UTC)
|
||||
}
|
||||
}
|
||||
|
||||
override fun compareDateTimeValues(expectedValue: String, actualValue: String): Boolean {
|
||||
val destinationDate = parseDateTime(actualValue)
|
||||
val expectedDate =
|
||||
LocalDateTime.parse(expectedValue, DateTimeFormatter.ofPattern(AIRBYTE_DATETIME_FORMAT))
|
||||
// #13123 Normalization issue
|
||||
if (expectedDate.isBefore(brokenDate.toLocalDateTime())) {
|
||||
LOGGER.warn(
|
||||
"Validation is skipped due to known Normalization issue. Values older then 1583 year and with time zone stored wrongly(lose days)."
|
||||
)
|
||||
return true
|
||||
} else {
|
||||
return expectedDate == destinationDate
|
||||
}
|
||||
}
|
||||
|
||||
override fun compareDateValues(expectedValue: String, actualValue: String): Boolean {
|
||||
val destinationDate = parseDate(actualValue)
|
||||
val expectedDate =
|
||||
LocalDate.parse(expectedValue, DateTimeFormatter.ofPattern(AIRBYTE_DATE_FORMAT))
|
||||
return expectedDate == destinationDate
|
||||
}
|
||||
|
||||
override fun compareDateTimeWithTzValues(
|
||||
airbyteMessageValue: String,
|
||||
destinationValue: String
|
||||
): Boolean {
|
||||
// #13123 Normalization issue
|
||||
if (parseDestinationDateWithTz(destinationValue).isBefore(brokenDate)) {
|
||||
LOGGER.warn(
|
||||
"Validation is skipped due to known Normalization issue. Values older then 1583 year and with time zone stored wrongly(lose days)."
|
||||
)
|
||||
return true
|
||||
} else {
|
||||
return super.compareDateTimeWithTzValues(airbyteMessageValue, destinationValue)
|
||||
}
|
||||
}
|
||||
|
||||
private val brokenDate: ZonedDateTime
|
||||
// #13123 Normalization issue
|
||||
get() = ZonedDateTime.of(1583, 1, 1, 0, 0, 0, 0, ZoneOffset.UTC)
|
||||
|
||||
override fun compareObjects(expectedObject: JsonNode, actualObject: JsonNode) {
|
||||
val actualJsonNode =
|
||||
(if (actualObject.isTextual) deserialize(actualObject.textValue()) else actualObject)
|
||||
super.compareObjects(expectedObject, actualJsonNode)
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val LOGGER: Logger = LoggerFactory.getLogger(BigQueryTestDataComparator::class.java)
|
||||
private const val BIGQUERY_DATETIME_FORMAT = "yyyy-MM-dd'T'HH:mm:ss'Z'"
|
||||
}
|
||||
}
|
||||
@@ -1,23 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.load.test.util.ConfigurationUpdater
|
||||
import io.airbyte.cdk.load.test.util.DefaultNamespaceResult
|
||||
|
||||
const val DEFAULT_NAMESPACE_PLACEHOLDER = "DEFAULT_NAMESPACE_PLACEHOLDER"
|
||||
|
||||
object BigqueryConfigUpdater : ConfigurationUpdater {
|
||||
override fun update(config: String): String = config
|
||||
|
||||
override fun setDefaultNamespace(
|
||||
config: String,
|
||||
defaultNamespace: String
|
||||
): DefaultNamespaceResult =
|
||||
DefaultNamespaceResult(
|
||||
config.replace(DEFAULT_NAMESPACE_PLACEHOLDER, defaultNamespace),
|
||||
defaultNamespace
|
||||
)
|
||||
}
|
||||
@@ -1,190 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.fasterxml.jackson.databind.node.ArrayNode
|
||||
import com.google.cloud.bigquery.FieldValue
|
||||
import com.google.cloud.bigquery.LegacySQLTypeName
|
||||
import com.google.cloud.bigquery.StandardTableDefinition
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import io.airbyte.cdk.command.ConfigurationSpecification
|
||||
import io.airbyte.cdk.load.command.DestinationStream
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
import io.airbyte.cdk.load.data.BooleanValue
|
||||
import io.airbyte.cdk.load.data.DateValue
|
||||
import io.airbyte.cdk.load.data.IntegerValue
|
||||
import io.airbyte.cdk.load.data.NullValue
|
||||
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.TimeWithoutTimezoneValue
|
||||
import io.airbyte.cdk.load.data.TimestampWithTimezoneValue
|
||||
import io.airbyte.cdk.load.data.TimestampWithoutTimezoneValue
|
||||
import io.airbyte.cdk.load.data.json.toAirbyteValue
|
||||
import io.airbyte.cdk.load.message.Meta
|
||||
import io.airbyte.cdk.load.test.util.DestinationDataDumper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
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.protocol.models.v0.AirbyteRecordMessageMetaChange
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMetaChange.Reason
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import java.math.BigDecimal
|
||||
import java.time.ZoneOffset
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
object BigqueryRawTableDataDumper : DestinationDataDumper {
|
||||
override fun dumpRecords(
|
||||
spec: ConfigurationSpecification,
|
||||
stream: DestinationStream
|
||||
): List<OutputRecord> {
|
||||
val config = BigqueryConfigurationFactory().make(spec as BigquerySpecification)
|
||||
val bigquery = BigqueryBeansFactory().getBigqueryClient(config)
|
||||
|
||||
val (_, rawTableName) =
|
||||
BigqueryRawTableNameGenerator(config).getTableName(stream.descriptor)
|
||||
|
||||
return bigquery.getTable(TableId.of(config.rawTableDataset, rawTableName))?.let { table ->
|
||||
val bigquerySchema = table.getDefinition<StandardTableDefinition>().schema!!
|
||||
table.list(bigquerySchema).iterateAll().map { row ->
|
||||
OutputRecord(
|
||||
rawId = row[Meta.COLUMN_NAME_AB_RAW_ID].stringValue,
|
||||
extractedAt =
|
||||
row[Meta.COLUMN_NAME_AB_EXTRACTED_AT].timestampInstant.toEpochMilli(),
|
||||
// loadedAt is nullable (e.g. if we disabled T+D, then it will always be null)
|
||||
loadedAt =
|
||||
row[Meta.COLUMN_NAME_AB_LOADED_AT].mapNotNull {
|
||||
it.timestampInstant.toEpochMilli()
|
||||
},
|
||||
generationId = row[Meta.COLUMN_NAME_AB_GENERATION_ID].longValue,
|
||||
data =
|
||||
row[Meta.COLUMN_NAME_DATA].stringValue.deserializeToNode().toAirbyteValue(),
|
||||
airbyteMeta = stringToMeta(row[Meta.COLUMN_NAME_AB_META].stringValue),
|
||||
)
|
||||
}
|
||||
}
|
||||
?: run {
|
||||
logger.warn {
|
||||
"Raw table does not exist: ${config.rawTableDataset}.$rawTableName. Returning empty list."
|
||||
}
|
||||
emptyList()
|
||||
}
|
||||
}
|
||||
|
||||
override fun dumpFile(
|
||||
spec: ConfigurationSpecification,
|
||||
stream: DestinationStream
|
||||
): Map<String, String> {
|
||||
throw NotImplementedError("Bigquery doesn't support file transfer")
|
||||
}
|
||||
}
|
||||
|
||||
object BigqueryFinalTableDataDumper : DestinationDataDumper {
|
||||
override fun dumpRecords(
|
||||
spec: ConfigurationSpecification,
|
||||
stream: DestinationStream
|
||||
): List<OutputRecord> {
|
||||
val config = BigqueryConfigurationFactory().make(spec as BigquerySpecification)
|
||||
val bigquery = BigqueryBeansFactory().getBigqueryClient(config)
|
||||
|
||||
val (datasetName, finalTableName) =
|
||||
BigqueryFinalTableNameGenerator(config).getTableName(stream.descriptor)
|
||||
|
||||
return bigquery.getTable(TableId.of(datasetName, finalTableName))?.let { table ->
|
||||
val bigquerySchema = table.getDefinition<StandardTableDefinition>().schema!!
|
||||
table.list(bigquerySchema).iterateAll().map { row ->
|
||||
val valuesMap: LinkedHashMap<String, AirbyteValue> =
|
||||
bigquerySchema.fields
|
||||
.filter { field -> !Meta.COLUMN_NAMES.contains(field.name) }
|
||||
.associateTo(linkedMapOf()) { field ->
|
||||
val value: FieldValue = row[field.name]
|
||||
val airbyteValue =
|
||||
value.mapNotNull {
|
||||
when (field.type) {
|
||||
LegacySQLTypeName.BOOLEAN -> BooleanValue(it.booleanValue)
|
||||
LegacySQLTypeName.BIGNUMERIC -> NumberValue(it.numericValue)
|
||||
LegacySQLTypeName.FLOAT ->
|
||||
NumberValue(BigDecimal(it.doubleValue))
|
||||
LegacySQLTypeName.NUMERIC -> NumberValue(it.numericValue)
|
||||
LegacySQLTypeName.INTEGER -> IntegerValue(it.longValue)
|
||||
LegacySQLTypeName.STRING -> StringValue(it.stringValue)
|
||||
// TODO check these
|
||||
LegacySQLTypeName.DATE -> DateValue(it.stringValue)
|
||||
LegacySQLTypeName.DATETIME ->
|
||||
TimestampWithoutTimezoneValue(it.stringValue)
|
||||
LegacySQLTypeName.TIME ->
|
||||
TimeWithoutTimezoneValue(it.stringValue)
|
||||
LegacySQLTypeName.TIMESTAMP ->
|
||||
TimestampWithTimezoneValue(
|
||||
it.timestampInstant.atOffset(ZoneOffset.UTC)
|
||||
)
|
||||
LegacySQLTypeName.JSON ->
|
||||
it.stringValue.deserializeToNode().toAirbyteValue()
|
||||
else ->
|
||||
throw UnsupportedOperationException(
|
||||
"Bigquery data dumper doesn't know how to dump type ${field.type} with value $it"
|
||||
)
|
||||
}
|
||||
}
|
||||
field.name to (airbyteValue ?: NullValue)
|
||||
}
|
||||
OutputRecord(
|
||||
rawId = row[Meta.COLUMN_NAME_AB_RAW_ID].stringValue,
|
||||
extractedAt =
|
||||
row[Meta.COLUMN_NAME_AB_EXTRACTED_AT].timestampInstant.toEpochMilli(),
|
||||
loadedAt = null,
|
||||
generationId = row[Meta.COLUMN_NAME_AB_GENERATION_ID].longValue,
|
||||
data = ObjectValue(valuesMap),
|
||||
airbyteMeta = stringToMeta(row[Meta.COLUMN_NAME_AB_META].stringValue),
|
||||
)
|
||||
}
|
||||
}
|
||||
?: run {
|
||||
logger.warn {
|
||||
"Final table does not exist: $datasetName.$finalTableName. Returning empty list."
|
||||
}
|
||||
return emptyList()
|
||||
}
|
||||
}
|
||||
|
||||
override fun dumpFile(
|
||||
spec: ConfigurationSpecification,
|
||||
stream: DestinationStream
|
||||
): Map<String, String> {
|
||||
throw NotImplementedError("Bigquery doesn't support file transfer")
|
||||
}
|
||||
}
|
||||
|
||||
fun stringToMeta(metaAsString: String): OutputRecord.Meta {
|
||||
val metaJson = Jsons.readTree(metaAsString)
|
||||
|
||||
val changes =
|
||||
(metaJson["changes"] as ArrayNode).map { change ->
|
||||
Meta.Change(
|
||||
field = change["field"].textValue(),
|
||||
change =
|
||||
AirbyteRecordMessageMetaChange.Change.fromValue(change["change"].textValue()),
|
||||
reason = Reason.fromValue(change["reason"].textValue()),
|
||||
)
|
||||
}
|
||||
|
||||
return OutputRecord.Meta(
|
||||
changes = changes,
|
||||
syncId = metaJson["sync_id"].longValue(),
|
||||
)
|
||||
}
|
||||
|
||||
fun <T> FieldValue.mapNotNull(f: (FieldValue) -> T): T? {
|
||||
return if (this.isNull) {
|
||||
null
|
||||
} else {
|
||||
f(this)
|
||||
}
|
||||
}
|
||||
@@ -1,126 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import io.airbyte.cdk.load.test.util.DestinationCleaner
|
||||
import io.airbyte.cdk.load.test.util.IntegrationTest
|
||||
import io.github.oshai.kotlinlogging.KotlinLogging
|
||||
import kotlinx.coroutines.Dispatchers
|
||||
import kotlinx.coroutines.launch
|
||||
import kotlinx.coroutines.runBlocking
|
||||
|
||||
private val logger = KotlinLogging.logger {}
|
||||
|
||||
object BigqueryDestinationCleaner : DestinationCleaner {
|
||||
private val actualCleaner =
|
||||
BigqueryDestinationCleanerInstance(
|
||||
BigQueryDestinationTestUtils.standardInsertRawOverrideConfig
|
||||
)
|
||||
.compose(
|
||||
BigqueryDestinationCleanerInstance(
|
||||
BigQueryDestinationTestUtils.standardInsertConfig
|
||||
)
|
||||
)
|
||||
|
||||
override fun cleanup() {
|
||||
// only run the cleaner sometimes - our nightlies will do this enough of the time
|
||||
// that we have a reasonably clean destination.
|
||||
// bigquery sets pretty harsh rate limits on some of the stuff the cleaner does.
|
||||
// (would be really nice if we stuck this in a cron somewhere + trigger it weekly,
|
||||
// but this is fine for now)
|
||||
if (Math.random() < 0.1) {
|
||||
actualCleaner.cleanup()
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
class BigqueryDestinationCleanerInstance(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)
|
||||
// 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.
|
||||
while (true) {
|
||||
launch {
|
||||
rawTables.values.forEach { table ->
|
||||
val tableName = table.tableId.table
|
||||
// in raw tables, we embed the namespace into the table name.
|
||||
// so we have to call isNamespaceOld on the table name.
|
||||
if (
|
||||
IntegrationTest.isNamespaceOld(
|
||||
tableName,
|
||||
retentionDays = RETENTION_DAYS
|
||||
)
|
||||
) {
|
||||
launch {
|
||||
logger.info { "Deleting table ${table.tableId}" }
|
||||
try {
|
||||
table.delete()
|
||||
} catch (e: BigQueryException) {
|
||||
// ignore exception
|
||||
// e.g. someone else might be running tests at the same time,
|
||||
// and deleted this table before we got to it
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (rawTables.hasNextPage()) {
|
||||
rawTables = rawTables.nextPage
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
logger.info { "Cleaning up old datasets in ${config.projectId}" }
|
||||
var datasets = bigquery.listDatasets(config.projectId)
|
||||
while (true) {
|
||||
launch {
|
||||
datasets.values.forEach { dataset ->
|
||||
if (
|
||||
dataset != null &&
|
||||
IntegrationTest.isNamespaceOld(
|
||||
dataset.datasetId.dataset,
|
||||
retentionDays = RETENTION_DAYS
|
||||
)
|
||||
) {
|
||||
launch {
|
||||
logger.info { "Deleting dataset ${dataset.datasetId}" }
|
||||
try {
|
||||
dataset.delete(BigQuery.DatasetDeleteOption.deleteContents())
|
||||
} catch (e: BigQueryException) {
|
||||
// ignore exception.
|
||||
// there are some test-generated datasets that our test user
|
||||
// doesn't have permissions on, for... some reason
|
||||
// or maybe someone else is running tests at the same time as
|
||||
// us, and we're racing to delete these datasets.
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (datasets.hasNextPage()) {
|
||||
datasets = datasets.nextPage
|
||||
} else {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
companion object {
|
||||
// set a more aggressive retention policy.
|
||||
// bigquery is _really_ slow at listing datasets/tables.
|
||||
const val RETENTION_DAYS = 7L
|
||||
}
|
||||
}
|
||||
@@ -1,9 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.load.spec.SpecTest
|
||||
|
||||
class BigquerySpecTest : SpecTest()
|
||||
@@ -1,345 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
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
|
||||
import io.airbyte.cdk.load.write.SchematizedNestedValueBehavior
|
||||
import io.airbyte.cdk.load.write.SimpleValueBehavior
|
||||
import io.airbyte.cdk.load.write.StronglyTyped
|
||||
import io.airbyte.cdk.load.write.UnionBehavior
|
||||
import io.airbyte.cdk.load.write.Untyped
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils.GCS_STAGING_CONFIG
|
||||
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 org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.assertAll
|
||||
|
||||
abstract class BigqueryWriteTest(
|
||||
configContents: String,
|
||||
dataDumper: DestinationDataDumper,
|
||||
expectedRecordMapper: ExpectedRecordMapper,
|
||||
isStreamSchemaRetroactive: Boolean,
|
||||
preserveUndeclaredFields: Boolean,
|
||||
dedupBehavior: DedupBehavior?,
|
||||
nullEqualsUnset: Boolean,
|
||||
allTypesBehavior: AllTypesBehavior,
|
||||
) :
|
||||
BasicFunctionalityIntegrationTest(
|
||||
configContents = configContents,
|
||||
BigquerySpecification::class.java,
|
||||
dataDumper,
|
||||
BigqueryDestinationCleaner,
|
||||
recordMangler = expectedRecordMapper,
|
||||
isStreamSchemaRetroactive = isStreamSchemaRetroactive,
|
||||
dedupBehavior = dedupBehavior,
|
||||
stringifySchemalessObjects = false,
|
||||
schematizedObjectBehavior = SchematizedNestedValueBehavior.PASS_THROUGH,
|
||||
schematizedArrayBehavior = SchematizedNestedValueBehavior.PASS_THROUGH,
|
||||
unionBehavior = UnionBehavior.PASS_THROUGH,
|
||||
preserveUndeclaredFields = preserveUndeclaredFields,
|
||||
supportFileTransfer = false,
|
||||
commitDataIncrementally = false,
|
||||
allTypesBehavior = allTypesBehavior,
|
||||
nullEqualsUnset = nullEqualsUnset,
|
||||
configUpdater = BigqueryConfigUpdater,
|
||||
additionalMicronautEnvs = additionalMicronautEnvs,
|
||||
)
|
||||
|
||||
abstract class BigqueryRawTablesWriteTest(
|
||||
configContents: String,
|
||||
) :
|
||||
BigqueryWriteTest(
|
||||
configContents = configContents,
|
||||
BigqueryRawTableDataDumper,
|
||||
UncoercedExpectedRecordMapper,
|
||||
isStreamSchemaRetroactive = false,
|
||||
preserveUndeclaredFields = true,
|
||||
dedupBehavior = null,
|
||||
nullEqualsUnset = false,
|
||||
Untyped,
|
||||
)
|
||||
|
||||
abstract class BigqueryTDWriteTest(configContents: String) :
|
||||
BigqueryWriteTest(
|
||||
configContents = configContents,
|
||||
BigqueryFinalTableDataDumper,
|
||||
ColumnNameModifyingMapper(BigqueryColumnNameGenerator())
|
||||
.compose(RootLevelTimestampsToUtcMapper)
|
||||
.compose(TypingDedupingMetaChangeMapper)
|
||||
.compose(IntegralNumberRecordMapper),
|
||||
isStreamSchemaRetroactive = true,
|
||||
preserveUndeclaredFields = false,
|
||||
dedupBehavior = DedupBehavior(),
|
||||
nullEqualsUnset = true,
|
||||
StronglyTyped(
|
||||
convertAllValuesToString = true,
|
||||
topLevelFloatLosesPrecision = true,
|
||||
nestedFloatLosesPrecision = true,
|
||||
integerCanBeLarge = false,
|
||||
numberCanBeLarge = false,
|
||||
timeWithTimezoneBehavior = SimpleValueBehavior.PASS_THROUGH,
|
||||
),
|
||||
) {
|
||||
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 :
|
||||
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()
|
||||
}
|
||||
}
|
||||
|
||||
class StandardInsert : BigqueryTDWriteTest(BigQueryDestinationTestUtils.standardInsertConfig) {
|
||||
@Test
|
||||
override fun testDedup() {
|
||||
super.testDedup()
|
||||
}
|
||||
}
|
||||
|
||||
class GcsRawOverrideDisableTd :
|
||||
BigqueryRawTablesWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
configFile = GCS_STAGING_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
disableTypingDeduping = true,
|
||||
),
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
|
||||
class GcsRawOverride :
|
||||
BigqueryTDWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(
|
||||
configFile = GCS_STAGING_CONFIG,
|
||||
rawDatasetId = RAW_DATASET_OVERRIDE,
|
||||
),
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
|
||||
class Gcs :
|
||||
BigqueryTDWriteTest(
|
||||
BigQueryDestinationTestUtils.createConfig(configFile = GCS_STAGING_CONFIG)
|
||||
) {
|
||||
@Test
|
||||
override fun testBasicWrite() {
|
||||
super.testBasicWrite()
|
||||
}
|
||||
}
|
||||
@@ -1,58 +0,0 @@
|
||||
/*
|
||||
* Copyright (c) 2025 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import io.airbyte.cdk.load.data.AirbyteType
|
||||
import io.airbyte.cdk.load.data.AirbyteValue
|
||||
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.test.util.ExpectedRecordMapper
|
||||
import io.airbyte.cdk.load.test.util.OutputRecord
|
||||
|
||||
/**
|
||||
* In nested JSON fields, bigquery converts integral numbers to integers. For example, if you try to
|
||||
* store `{"foo": 5.0}` in a JSON column, bigquery will actually store `{"foo": 5}`.
|
||||
*
|
||||
* (however, we don't want to modify root-level fields, because those are actual NUMERIC/INTEGER
|
||||
* columns).
|
||||
*/
|
||||
object IntegralNumberRecordMapper : ExpectedRecordMapper {
|
||||
override fun mapRecord(expectedRecord: OutputRecord, schema: AirbyteType): OutputRecord {
|
||||
val mappedData = mapNestedIntegralNumberToInteger(expectedRecord.data, level = 0)
|
||||
return expectedRecord.copy(data = mappedData as ObjectValue)
|
||||
}
|
||||
|
||||
private fun mapNestedIntegralNumberToInteger(value: AirbyteValue, level: Int): AirbyteValue =
|
||||
when (value) {
|
||||
is NumberValue -> {
|
||||
// level 0 is the root object
|
||||
// level 1 is any root-level fields
|
||||
// level 2 and above is anything inside a subobject/array
|
||||
if (level > 1) {
|
||||
// can't use `.equals`, because that also checks for scale (i.e. 2.0 != 2)
|
||||
if (value.value.toBigInteger().toBigDecimal().compareTo(value.value) == 0) {
|
||||
IntegerValue(value.value.toBigInteger())
|
||||
} else {
|
||||
value
|
||||
}
|
||||
} else {
|
||||
value
|
||||
}
|
||||
}
|
||||
is ArrayValue ->
|
||||
ArrayValue(
|
||||
value.values.map { mapNestedIntegralNumberToInteger(it, level = level + 1) }
|
||||
)
|
||||
is ObjectValue ->
|
||||
ObjectValue(
|
||||
value.values.mapValuesTo(linkedMapOf()) { (_, v) ->
|
||||
mapNestedIntegralNumberToInteger(v, level = level + 1)
|
||||
}
|
||||
)
|
||||
else -> value
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,213 @@
|
||||
/*
|
||||
* Copyright (c) 2024 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
|
||||
package io.airbyte.integrations.destination.bigquery.operation
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.BigQueryException
|
||||
import com.google.cloud.bigquery.DatasetId
|
||||
import com.google.cloud.bigquery.DatasetInfo
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import com.google.cloud.bigquery.TableResult
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteMessage
|
||||
import io.airbyte.cdk.integrations.destination.async.model.PartialAirbyteRecordMessage
|
||||
import io.airbyte.commons.json.Jsons
|
||||
import io.airbyte.commons.string.Strings
|
||||
import io.airbyte.integrations.base.destination.operation.AbstractStreamOperation.Companion.TMP_TABLE_SUFFIX
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.ImportType
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamId
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestination
|
||||
import io.airbyte.integrations.destination.bigquery.formatter.BigQueryRecordFormatter
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGenerator
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQuerySqlGeneratorIntegrationTest
|
||||
import io.airbyte.protocol.models.v0.AirbyteMessage.Type
|
||||
import io.airbyte.protocol.models.v0.AirbyteRecordMessageMeta
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Path
|
||||
import java.util.Optional
|
||||
import java.util.stream.Stream
|
||||
import kotlin.test.assertEquals
|
||||
import org.junit.jupiter.api.AfterEach
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.assertThrows
|
||||
import org.junit.jupiter.api.parallel.Execution
|
||||
import org.junit.jupiter.api.parallel.ExecutionMode
|
||||
|
||||
/**
|
||||
* Extremely barebones integration test for the direct inserts storage op. We should eventually:
|
||||
* * Make something similar for the GCS storage op
|
||||
* * Genericize this and put it in the CDK
|
||||
* * Add assertions for all the columns, not just airbyte_data
|
||||
* * Actually test all the methods on StorageOperation
|
||||
*/
|
||||
@Execution(ExecutionMode.CONCURRENT)
|
||||
class BigQueryDirectLoadingStorageOperationTest {
|
||||
private val randomString = Strings.addRandomSuffix("", "", 10)
|
||||
private val streamId =
|
||||
StreamId(
|
||||
finalNamespace = "final_namespace_$randomString",
|
||||
finalName = "final_name_$randomString",
|
||||
rawNamespace = "raw_namespace_$randomString",
|
||||
rawName = "raw_name_$randomString",
|
||||
originalNamespace = "original_namespace_$randomString",
|
||||
originalName = "original_name_$randomString",
|
||||
)
|
||||
private val streamConfig =
|
||||
StreamConfig(
|
||||
streamId,
|
||||
ImportType.APPEND,
|
||||
emptyList(),
|
||||
Optional.empty(),
|
||||
LinkedHashMap(),
|
||||
GENERATION_ID,
|
||||
0,
|
||||
SYNC_ID,
|
||||
)
|
||||
|
||||
@BeforeEach
|
||||
fun setup() {
|
||||
bq.create(DatasetInfo.of(streamId.rawNamespace))
|
||||
}
|
||||
|
||||
@AfterEach
|
||||
fun teardown() {
|
||||
bq.delete(
|
||||
DatasetId.of(streamId.rawNamespace),
|
||||
BigQuery.DatasetDeleteOption.deleteContents()
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testTransferStage() {
|
||||
storageOperation.prepareStage(streamId, "")
|
||||
storageOperation.prepareStage(streamId, TMP_TABLE_SUFFIX)
|
||||
// Table is currently empty, so expect null generation.
|
||||
assertEquals(null, storageOperation.getStageGeneration(streamId, TMP_TABLE_SUFFIX))
|
||||
|
||||
// Write one record to the real raw table
|
||||
storageOperation.writeToStage(
|
||||
streamConfig,
|
||||
"",
|
||||
Stream.of(record(1)),
|
||||
)
|
||||
assertEquals(
|
||||
listOf("""{"record_number": 1}"""),
|
||||
// We write the raw data as a string column, not a JSON column, so use asText().
|
||||
dumpRawRecords("").map { it["_airbyte_data"].asText() },
|
||||
)
|
||||
|
||||
// And write one record to the temp final table
|
||||
storageOperation.writeToStage(
|
||||
streamConfig,
|
||||
TMP_TABLE_SUFFIX,
|
||||
Stream.of(record(2)),
|
||||
)
|
||||
assertEquals(
|
||||
listOf("""{"record_number": 2}"""),
|
||||
dumpRawRecords(TMP_TABLE_SUFFIX).map { it["_airbyte_data"].asText() },
|
||||
)
|
||||
assertEquals(GENERATION_ID, storageOperation.getStageGeneration(streamId, TMP_TABLE_SUFFIX))
|
||||
|
||||
// If we transfer the records, we should end up with 2 records in the real raw table.
|
||||
storageOperation.transferFromTempStage(streamId, TMP_TABLE_SUFFIX)
|
||||
assertEquals(
|
||||
listOf(
|
||||
"""{"record_number": 1}""",
|
||||
"""{"record_number": 2}""",
|
||||
),
|
||||
dumpRawRecords("")
|
||||
.sortedBy {
|
||||
Jsons.deserialize(it["_airbyte_data"].asText())["record_number"].asLong()
|
||||
}
|
||||
.map { it["_airbyte_data"].asText() },
|
||||
)
|
||||
|
||||
// After transferring the records to the real table, the temp table should no longer exist.
|
||||
assertEquals(404, assertThrows<BigQueryException> { dumpRawRecords(TMP_TABLE_SUFFIX) }.code)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testOverwriteStage() {
|
||||
// If we then create another temp raw table and _overwrite_ the real raw table,
|
||||
// we should end up with a single raw record.
|
||||
storageOperation.prepareStage(streamId, "")
|
||||
storageOperation.prepareStage(streamId, TMP_TABLE_SUFFIX)
|
||||
storageOperation.writeToStage(
|
||||
streamConfig,
|
||||
"",
|
||||
Stream.of(record(3)),
|
||||
)
|
||||
storageOperation.writeToStage(
|
||||
streamConfig,
|
||||
TMP_TABLE_SUFFIX,
|
||||
Stream.of(record(4)),
|
||||
)
|
||||
|
||||
storageOperation.overwriteStage(streamId, TMP_TABLE_SUFFIX)
|
||||
|
||||
assertEquals(
|
||||
listOf("""{"record_number": 4}"""),
|
||||
dumpRawRecords("").map { it["_airbyte_data"].asText() },
|
||||
)
|
||||
assertEquals(404, assertThrows<BigQueryException> { dumpRawRecords(TMP_TABLE_SUFFIX) }.code)
|
||||
}
|
||||
|
||||
private fun dumpRawRecords(suffix: String): List<JsonNode> {
|
||||
val result: TableResult =
|
||||
bq.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT * FROM " + streamId.rawTableId(BigQuerySqlGenerator.QUOTE, suffix)
|
||||
),
|
||||
)
|
||||
return BigQuerySqlGeneratorIntegrationTest.toJsonRecords(result)
|
||||
}
|
||||
|
||||
private fun record(recordNumber: Int): PartialAirbyteMessage {
|
||||
val serializedData = """{"record_number": $recordNumber}"""
|
||||
return PartialAirbyteMessage()
|
||||
.withType(Type.RECORD)
|
||||
.withSerialized(serializedData)
|
||||
.withRecord(
|
||||
PartialAirbyteRecordMessage()
|
||||
.withNamespace(streamId.originalNamespace)
|
||||
.withStream(streamId.originalName)
|
||||
.withEmittedAt(10_000)
|
||||
.withMeta(
|
||||
AirbyteRecordMessageMeta()
|
||||
.withChanges(emptyList())
|
||||
.withAdditionalProperty(
|
||||
JavaBaseConstants.AIRBYTE_META_SYNC_ID_KEY,
|
||||
SYNC_ID,
|
||||
),
|
||||
)
|
||||
.withData(Jsons.deserialize(serializedData)),
|
||||
)
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val config =
|
||||
Jsons.deserialize(Files.readString(Path.of("secrets/credentials-gcs-staging.json")))
|
||||
private val bq = BigQueryDestination.getBigQuery(config)
|
||||
private val projectId = config.get(BigQueryConsts.CONFIG_PROJECT_ID).asText()
|
||||
private val datasetLocation = config.get(BigQueryConsts.CONFIG_DATASET_LOCATION).asText()
|
||||
private val storageOperation =
|
||||
BigQueryDirectLoadingStorageOperation(
|
||||
bq,
|
||||
15,
|
||||
BigQueryRecordFormatter(),
|
||||
BigQuerySqlGenerator(projectId, datasetLocation),
|
||||
BigQueryDestinationHandler(bq, datasetLocation),
|
||||
datasetLocation,
|
||||
)
|
||||
|
||||
private const val SYNC_ID = 12L
|
||||
private const val GENERATION_ID = 42L
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,437 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode
|
||||
import com.google.cloud.bigquery.BigQuery
|
||||
import com.google.cloud.bigquery.DatasetId
|
||||
import com.google.cloud.bigquery.QueryJobConfiguration
|
||||
import com.google.cloud.bigquery.RangePartitioning
|
||||
import com.google.cloud.bigquery.StandardTableDefinition
|
||||
import com.google.cloud.bigquery.TableId
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.commons.json.Jsons
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.BaseTypingDedupingTest
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.SqlGenerator
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamId
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getBigQuery
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestinationTestUtils
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId
|
||||
import io.airbyte.protocol.models.v0.*
|
||||
import io.airbyte.workers.exception.TestHarnessException
|
||||
import java.io.IOException
|
||||
import java.nio.file.Path
|
||||
import java.util.function.Function
|
||||
import java.util.stream.Collectors
|
||||
import kotlin.test.assertEquals
|
||||
import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.Disabled
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
abstract class AbstractBigQueryTypingDedupingTest : BaseTypingDedupingTest() {
|
||||
private var bq: BigQuery? = null
|
||||
|
||||
protected abstract val configPath: String
|
||||
get
|
||||
|
||||
@Throws(IOException::class)
|
||||
public override fun generateConfig(): JsonNode? {
|
||||
val datasetId = "typing_deduping_default_dataset$uniqueSuffix"
|
||||
val stagingPath = "test_path$uniqueSuffix"
|
||||
val config =
|
||||
BigQueryDestinationTestUtils.createConfig(Path.of(configPath), datasetId, stagingPath)
|
||||
bq = getBigQuery(config!!)
|
||||
return config
|
||||
}
|
||||
|
||||
override val imageName: String
|
||||
get() = "airbyte/destination-bigquery:dev"
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
override fun dumpRawTableRecords(streamNamespace: String?, streamName: String): List<JsonNode> {
|
||||
var streamNamespace = streamNamespace
|
||||
if (streamNamespace == null) {
|
||||
streamNamespace = getDatasetId(config!!)
|
||||
}
|
||||
val result =
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT * FROM " +
|
||||
rawDataset +
|
||||
"." +
|
||||
StreamId.concatenateRawTableName(streamNamespace, streamName)
|
||||
)
|
||||
)
|
||||
return BigQuerySqlGeneratorIntegrationTest.Companion.toJsonRecords(result)
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
override fun dumpFinalTableRecords(
|
||||
streamNamespace: String?,
|
||||
streamName: String
|
||||
): List<JsonNode> {
|
||||
var streamNamespace = streamNamespace
|
||||
if (streamNamespace == null) {
|
||||
streamNamespace = getDatasetId(config!!)
|
||||
}
|
||||
val result =
|
||||
bq!!.query(QueryJobConfiguration.of("SELECT * FROM $streamNamespace.$streamName"))
|
||||
return BigQuerySqlGeneratorIntegrationTest.Companion.toJsonRecords(result)
|
||||
}
|
||||
|
||||
override fun teardownStreamAndNamespace(streamNamespace: String?, streamName: String) {
|
||||
var streamNamespace = streamNamespace
|
||||
if (streamNamespace == null) {
|
||||
streamNamespace = getDatasetId(config!!)
|
||||
}
|
||||
// bq.delete simply returns false if the table/schema doesn't exist (e.g. if the connector
|
||||
// failed to
|
||||
// create it)
|
||||
// so we don't need to do any existence checks here.
|
||||
bq!!.delete(
|
||||
TableId.of(rawDataset, StreamId.concatenateRawTableName(streamNamespace, streamName))
|
||||
)
|
||||
bq!!.delete(DatasetId.of(streamNamespace), BigQuery.DatasetDeleteOption.deleteContents())
|
||||
}
|
||||
|
||||
override val sqlGenerator: SqlGenerator
|
||||
get() = BigQuerySqlGenerator(config!![BigQueryConsts.CONFIG_PROJECT_ID].asText(), null)
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
fun testV1V2Migration() {
|
||||
val catalog =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
java.util.List.of<ConfiguredAirbyteStream>(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.FULL_REFRESH)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncId(42L)
|
||||
.withGenerationId(43L)
|
||||
.withMinimumGenerationId(0L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(Companion.SCHEMA)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
// First sync
|
||||
val messages1 = readMessages("dat/sync1_messages.jsonl")
|
||||
|
||||
runSync(
|
||||
catalog,
|
||||
messages1,
|
||||
"airbyte/destination-bigquery:1.10.2",
|
||||
{ config: JsonNode? ->
|
||||
// Defensive to avoid weird behaviors or test failures if the original config is
|
||||
// being altered by
|
||||
// another thread, thanks jackson for a mutable JsonNode
|
||||
val copiedConfig: JsonNode = Jsons.clone<JsonNode>(config!!)
|
||||
if (config is ObjectNode) {
|
||||
// Opt out of T+D to run old V1 sync
|
||||
(copiedConfig as ObjectNode).put("use_1s1t_format", false)
|
||||
}
|
||||
copiedConfig
|
||||
}
|
||||
)
|
||||
|
||||
// The record differ code is already adapted to V2 columns format, use the post V2 sync
|
||||
// to verify that append mode preserved all the raw records and final records.
|
||||
|
||||
// Second sync
|
||||
val messages2 = readMessages("dat/sync2_messages.jsonl")
|
||||
|
||||
runSync(catalog, messages2)
|
||||
|
||||
val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_v1v2_raw.jsonl")
|
||||
val expectedFinalRecords2 =
|
||||
readRecords("dat/sync2_expectedrecords_v1v2_fullrefresh_append_final.jsonl")
|
||||
verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison())
|
||||
}
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
open fun testRemovingPKNonNullIndexes() {
|
||||
val catalog =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
java.util.List.of<ConfiguredAirbyteStream>(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND_DEDUP)
|
||||
.withSyncId(42L)
|
||||
.withGenerationId(43L)
|
||||
.withMinimumGenerationId(0L)
|
||||
.withPrimaryKey(
|
||||
java.util.List.of<List<String>>(
|
||||
listOf<String>("id1"),
|
||||
listOf<String>("id2")
|
||||
)
|
||||
)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(Companion.SCHEMA)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
// First sync
|
||||
val messages = readMessages("dat/sync_null_pk.jsonl")
|
||||
val e =
|
||||
Assertions.assertThrows(TestHarnessException::class.java) {
|
||||
runSync(catalog, messages, "airbyte/destination-bigquery:2.0.20")
|
||||
} // this version introduced non-null PKs to the final tables
|
||||
|
||||
// ideally we would assert on the logged content of the original exception within e, but
|
||||
// that is
|
||||
// proving to be tricky
|
||||
|
||||
// Second sync
|
||||
runSync(catalog, messages) // does not throw with latest version
|
||||
Assertions.assertEquals(
|
||||
1,
|
||||
dumpFinalTableRecords(streamNamespace, streamName).toTypedArray().size
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
fun testAirbyteMetaAndGenerationIdMigration() {
|
||||
val catalog =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
java.util.List.of<ConfiguredAirbyteStream>(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.FULL_REFRESH)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncId(42L)
|
||||
.withGenerationId(43L)
|
||||
.withMinimumGenerationId(0L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(Companion.SCHEMA)
|
||||
)
|
||||
)
|
||||
)
|
||||
|
||||
// First sync
|
||||
val messages1 = readMessages("dat/sync1_messages.jsonl")
|
||||
// We don't want to send a stream status message, because this version of
|
||||
// destination-bigquery will
|
||||
// crash.
|
||||
runSync(
|
||||
catalog,
|
||||
messages1,
|
||||
"airbyte/destination-bigquery:2.4.20",
|
||||
Function.identity(),
|
||||
null
|
||||
)
|
||||
|
||||
// Second sync
|
||||
val messages2 = readMessages("dat/sync2_messages.jsonl")
|
||||
runSync(catalog, messages2)
|
||||
|
||||
// The first 5 records in these files were written by the old version, and have
|
||||
// several differences with the new records:
|
||||
// In raw tables: no _airbyte_meta or _airbyte_generation_id at all
|
||||
// In final tables: no generation ID, and airbyte_meta still uses the old `{errors: [...]}`
|
||||
// structure
|
||||
// So modify the expected records to reflect those differences.
|
||||
val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_raw.jsonl")
|
||||
for (i in 0..4) {
|
||||
val record = expectedRawRecords2[i] as ObjectNode
|
||||
record.remove(JavaBaseConstants.COLUMN_NAME_AB_META)
|
||||
record.remove(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID)
|
||||
}
|
||||
val expectedFinalRecords2 =
|
||||
readRecords("dat/sync2_expectedrecords_fullrefresh_append_final.jsonl")
|
||||
for (i in 0..4) {
|
||||
val record = expectedFinalRecords2[i] as ObjectNode
|
||||
record.set<JsonNode>(
|
||||
JavaBaseConstants.COLUMN_NAME_AB_META,
|
||||
deserialize(
|
||||
"""
|
||||
{"errors": []}
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
record.remove(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID)
|
||||
}
|
||||
verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison())
|
||||
|
||||
// Verify that we didn't trigger a soft reset.
|
||||
// There should be two unique loaded_at values in the raw table.
|
||||
// (only do this if T+D is enabled to begin with; otherwise loaded_at will just be null)
|
||||
if (!disableFinalTableComparison()) {
|
||||
val actualRawRecords2 = dumpRawTableRecords(streamNamespace, streamName)
|
||||
val loadedAtValues =
|
||||
actualRawRecords2
|
||||
.stream()
|
||||
.map<JsonNode>(
|
||||
Function<JsonNode, JsonNode> { record: JsonNode ->
|
||||
record.get(JavaBaseConstants.COLUMN_NAME_AB_LOADED_AT)
|
||||
}
|
||||
)
|
||||
.collect(Collectors.toSet<JsonNode>())
|
||||
Assertions.assertEquals(
|
||||
2,
|
||||
loadedAtValues.size,
|
||||
"Expected two different values for loaded_at. If there is only 1 value, then we incorrectly triggered a soft reset. If there are more than 2, then something weird happened?"
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGenerationIdMigrationForOverwrite() {
|
||||
// First sync
|
||||
val catalog1 =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
listOf(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.FULL_REFRESH)
|
||||
.withDestinationSyncMode(DestinationSyncMode.OVERWRITE)
|
||||
.withSyncId(41L)
|
||||
.withGenerationId(42L)
|
||||
.withMinimumGenerationId(0L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(SCHEMA),
|
||||
),
|
||||
),
|
||||
)
|
||||
val messages1 = readMessages("dat/sync1_messages.jsonl")
|
||||
runSync(
|
||||
catalog1,
|
||||
messages1,
|
||||
"airbyte/destination-bigquery:2.4.20",
|
||||
// Old connector version can't handle TRACE messages; disable the
|
||||
// stream status message
|
||||
streamStatus = null,
|
||||
)
|
||||
|
||||
// Second sync
|
||||
val catalog2 =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
listOf(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.FULL_REFRESH)
|
||||
.withDestinationSyncMode(DestinationSyncMode.OVERWRITE)
|
||||
.withSyncId(42L)
|
||||
.withGenerationId(43L)
|
||||
.withMinimumGenerationId(43L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(SCHEMA),
|
||||
),
|
||||
),
|
||||
)
|
||||
val messages2 = readMessages("dat/sync2_messages.jsonl")
|
||||
runSync(catalog2, messages2)
|
||||
|
||||
val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_overwrite_raw.jsonl")
|
||||
val expectedFinalRecords2 =
|
||||
readRecords("dat/sync2_expectedrecords_fullrefresh_overwrite_final.jsonl")
|
||||
verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison())
|
||||
}
|
||||
|
||||
/**
|
||||
* In 2.10.0, we updated the default raw table partitioning to be a daily partition against
|
||||
* `_airbyte_extracted_at`. Verify that we didn't do anything weird. Additionally, verify that
|
||||
* we didn't repartition existing raw tables.
|
||||
*/
|
||||
@Test
|
||||
fun testRawTablePartitioningChange() {
|
||||
val catalog =
|
||||
ConfiguredAirbyteCatalog()
|
||||
.withStreams(
|
||||
listOf(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withSyncId(42L)
|
||||
.withGenerationId(42L)
|
||||
.withMinimumGenerationId(0L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withNamespace(streamNamespace)
|
||||
.withName(streamName)
|
||||
.withJsonSchema(SCHEMA),
|
||||
),
|
||||
),
|
||||
)
|
||||
val messages1 = readMessages("dat/sync1_messages.jsonl")
|
||||
runSync(
|
||||
catalog,
|
||||
messages1,
|
||||
"airbyte/destination-bigquery:2.9.3",
|
||||
)
|
||||
val messages2 = readMessages("dat/sync2_messages.jsonl")
|
||||
runSync(catalog, messages2)
|
||||
|
||||
// check that we have the correct data (i.e. a combination of sync1 + sync2)
|
||||
val expectedRawRecords2 = readRecords("dat/sync2_expectedrecords_raw.jsonl")
|
||||
val expectedFinalRecords2 =
|
||||
readRecords("dat/sync2_expectedrecords_fullrefresh_append_final.jsonl")
|
||||
fixGenerationId(expectedRawRecords2, expectedFinalRecords2, 42L)
|
||||
verifySyncResult(expectedRawRecords2, expectedFinalRecords2, disableFinalTableComparison())
|
||||
|
||||
// check that we still have the old generation_id partitioning
|
||||
val table =
|
||||
bq!!.getTable(
|
||||
rawDataset,
|
||||
StreamId.concatenateRawTableName(
|
||||
streamNamespace ?: getDatasetId(config!!),
|
||||
streamName
|
||||
)
|
||||
)
|
||||
assertEquals(
|
||||
RangePartitioning.newBuilder()
|
||||
.setField(JavaBaseConstants.COLUMN_NAME_AB_GENERATION_ID)
|
||||
.setRange(
|
||||
RangePartitioning.Range.newBuilder()
|
||||
.setStart(0L) // Bigquery allows a table to have up to 10_000 partitions.
|
||||
.setEnd(
|
||||
10000L
|
||||
) // Somewhat conservative estimate. This should avoid issues with
|
||||
// users running many merge refreshes.
|
||||
.setInterval(5L)
|
||||
.build()
|
||||
)
|
||||
.build(),
|
||||
table.getDefinition<StandardTableDefinition>().rangePartitioning,
|
||||
)
|
||||
}
|
||||
|
||||
// Disabling until we can safely fetch generation ID
|
||||
@Test
|
||||
@Disabled
|
||||
override fun interruptedOverwriteWithoutPriorData() {
|
||||
super.interruptedOverwriteWithoutPriorData()
|
||||
}
|
||||
|
||||
protected open val rawDataset: String
|
||||
/**
|
||||
* Subclasses using a config with a nonstandard raw table dataset should override this
|
||||
* method.
|
||||
*/
|
||||
get() = JavaBaseConstants.DEFAULT_AIRBYTE_INTERNAL_NAMESPACE
|
||||
}
|
||||
@@ -0,0 +1,31 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import org.junit.jupiter.api.Disabled
|
||||
|
||||
class BigQueryGcsRawOverrideDisableTypingDedupingTest : AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-disabletd-gcs-raw-override.json"
|
||||
|
||||
override val rawDataset: String
|
||||
get() = "overridden_raw_dataset"
|
||||
|
||||
override fun disableFinalTableComparison(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Throws(Exception::class)
|
||||
override fun testRemovingPKNonNullIndexes() {
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Throws(Exception::class)
|
||||
override fun identicalNameSimultaneousSync() {
|
||||
// TODO: create fixtures to verify how raw tables are affected. Base tests check for final
|
||||
// tables.
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,12 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
class BigQueryGcsRawOverrideTypingDedupingTest : AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-gcs-raw-override.json"
|
||||
|
||||
override val rawDataset: String
|
||||
get() = "overridden_raw_dataset"
|
||||
}
|
||||
@@ -0,0 +1,9 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
class BigQueryGcsTypingDedupingTest : AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-gcs.json"
|
||||
}
|
||||
@@ -0,0 +1,620 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.fasterxml.jackson.databind.JsonNode
|
||||
import com.fasterxml.jackson.databind.node.ObjectNode
|
||||
import com.google.cloud.bigquery.*
|
||||
import edu.umd.cs.findbugs.annotations.SuppressFBWarnings
|
||||
import io.airbyte.cdk.integrations.base.JavaBaseConstants
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.commons.json.Jsons.deserializeExact
|
||||
import io.airbyte.commons.json.Jsons.emptyObject
|
||||
import io.airbyte.commons.json.Jsons.jsonNode
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.*
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryConsts
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryDestination.Companion.getBigQuery
|
||||
import io.airbyte.integrations.destination.bigquery.migrators.BigQueryDestinationState
|
||||
import java.nio.file.Files
|
||||
import java.nio.file.Path
|
||||
import java.time.Duration
|
||||
import java.util.*
|
||||
import java.util.Map
|
||||
import java.util.stream.Collectors
|
||||
import kotlin.collections.LinkedHashMap
|
||||
import kotlin.collections.List
|
||||
import kotlin.collections.emptyList
|
||||
import kotlin.collections.indices
|
||||
import org.apache.commons.text.StringSubstitutor
|
||||
import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.BeforeAll
|
||||
import org.junit.jupiter.api.Disabled
|
||||
import org.junit.jupiter.api.Test
|
||||
import org.junit.jupiter.api.parallel.Execution
|
||||
import org.junit.jupiter.api.parallel.ExecutionMode
|
||||
import org.junit.jupiter.params.ParameterizedTest
|
||||
import org.junit.jupiter.params.provider.ValueSource
|
||||
import org.slf4j.Logger
|
||||
import org.slf4j.LoggerFactory
|
||||
|
||||
@Execution(ExecutionMode.CONCURRENT)
|
||||
class BigQuerySqlGeneratorIntegrationTest :
|
||||
BaseSqlGeneratorIntegrationTest<BigQueryDestinationState>() {
|
||||
override val sqlGenerator: BigQuerySqlGenerator
|
||||
get() = BigQuerySqlGenerator(projectId, datasetLocation)
|
||||
|
||||
override val destinationHandler: BigQueryDestinationHandler
|
||||
get() = BigQueryDestinationHandler(bq!!, "US")
|
||||
|
||||
override fun createNamespace(namespace: String) {
|
||||
bq!!.create(
|
||||
DatasetInfo.newBuilder(
|
||||
namespace
|
||||
) // This unfortunately doesn't delete the actual dataset after 3 days, but at least
|
||||
// we'll clear out
|
||||
// old tables automatically
|
||||
.setDefaultTableLifetime(Duration.ofDays(3).toMillis())
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
override fun createRawTable(streamId: StreamId) {
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
Map.of("raw_table_id", streamId.rawTableId(BigQuerySqlGenerator.QUOTE))
|
||||
)
|
||||
.replace(
|
||||
"""
|
||||
CREATE TABLE ${'$'}{raw_table_id} (
|
||||
_airbyte_raw_id STRING NOT NULL,
|
||||
_airbyte_data STRING NOT NULL,
|
||||
_airbyte_extracted_at TIMESTAMP NOT NULL,
|
||||
_airbyte_loaded_at TIMESTAMP,
|
||||
_airbyte_meta STRING,
|
||||
_airbyte_generation_id INTEGER
|
||||
) PARTITION BY (
|
||||
DATE_TRUNC(_airbyte_extracted_at, DAY)
|
||||
) CLUSTER BY _airbyte_loaded_at;
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun createV1RawTable(v1RawTable: StreamId) {
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
Map.of(
|
||||
"raw_table_id",
|
||||
v1RawTable.rawTableId(BigQuerySqlGenerator.QUOTE)
|
||||
)
|
||||
)
|
||||
.replace(
|
||||
"""
|
||||
CREATE TABLE ${'$'}{raw_table_id} (
|
||||
_airbyte_ab_id STRING NOT NULL,
|
||||
_airbyte_data STRING NOT NULL,
|
||||
_airbyte_emitted_at TIMESTAMP NOT NULL,
|
||||
) PARTITION BY (
|
||||
DATE_TRUNC(_airbyte_emitted_at, DAY)
|
||||
) CLUSTER BY _airbyte_emitted_at;
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
@SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE")
|
||||
override fun insertFinalTableRecords(
|
||||
includeCdcDeletedAt: Boolean,
|
||||
streamId: StreamId,
|
||||
suffix: String?,
|
||||
records: List<JsonNode>,
|
||||
generationId: Long
|
||||
) {
|
||||
val columnNames =
|
||||
if (includeCdcDeletedAt) FINAL_TABLE_COLUMN_NAMES_CDC else FINAL_TABLE_COLUMN_NAMES
|
||||
val cdcDeletedAtDecl = if (includeCdcDeletedAt) ",`_ab_cdc_deleted_at` TIMESTAMP" else ""
|
||||
val cdcDeletedAtName = if (includeCdcDeletedAt) ",`_ab_cdc_deleted_at`" else ""
|
||||
val recordsText =
|
||||
records
|
||||
.stream() // For each record, convert it to a string like "(rawId, extractedAt,
|
||||
// loadedAt, data)"
|
||||
.map { record: JsonNode ->
|
||||
columnNames
|
||||
.stream()
|
||||
.map { fieldName: String? -> record[fieldName] }
|
||||
.map { r: JsonNode? ->
|
||||
if (r == null) {
|
||||
return@map "NULL"
|
||||
}
|
||||
val stringContents =
|
||||
if (r.isTextual) {
|
||||
r.asText()
|
||||
} else {
|
||||
r.toString()
|
||||
}
|
||||
'"'.toString() +
|
||||
stringContents // Serialized json might contain backslashes and
|
||||
// double quotes. Escape them.
|
||||
.replace("\\", "\\\\")
|
||||
.replace("\"", "\\\"") +
|
||||
'"'
|
||||
}
|
||||
.collect(Collectors.joining(","))
|
||||
}
|
||||
.map { row: String -> "($row)" }
|
||||
.collect(Collectors.joining(","))
|
||||
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
Map.of(
|
||||
"final_table_id",
|
||||
streamId.finalTableId(BigQuerySqlGenerator.QUOTE, suffix!!),
|
||||
"cdc_deleted_at_name",
|
||||
cdcDeletedAtName,
|
||||
"cdc_deleted_at_decl",
|
||||
cdcDeletedAtDecl,
|
||||
"records",
|
||||
recordsText
|
||||
)
|
||||
)
|
||||
.replace( // Similar to insertRawTableRecords, some of these columns are
|
||||
// declared as string and wrapped in
|
||||
// parse_json().
|
||||
// There's also a bunch of casting, because bigquery doesn't coerce
|
||||
// strings to e.g. int
|
||||
"""
|
||||
insert into ${'$'}{final_table_id} (
|
||||
_airbyte_raw_id,
|
||||
_airbyte_extracted_at,
|
||||
_airbyte_meta,
|
||||
_airbyte_generation_id,
|
||||
`id1`,
|
||||
`id2`,
|
||||
`updated_at`,
|
||||
`struct`,
|
||||
`array`,
|
||||
`string`,
|
||||
`number`,
|
||||
`integer`,
|
||||
`boolean`,
|
||||
`timestamp_with_timezone`,
|
||||
`timestamp_without_timezone`,
|
||||
`time_with_timezone`,
|
||||
`time_without_timezone`,
|
||||
`date`,
|
||||
`unknown`
|
||||
${'$'}{cdc_deleted_at_name}
|
||||
)
|
||||
select
|
||||
_airbyte_raw_id,
|
||||
_airbyte_extracted_at,
|
||||
parse_json(_airbyte_meta),
|
||||
_airbyte_generation_id,
|
||||
cast(`id1` as int64),
|
||||
cast(`id2` as int64),
|
||||
`updated_at`,
|
||||
parse_json(`struct`),
|
||||
parse_json(`array`),
|
||||
`string`,
|
||||
cast(`number` as numeric),
|
||||
cast(`integer` as int64),
|
||||
cast(`boolean` as boolean),
|
||||
`timestamp_with_timezone`,
|
||||
`timestamp_without_timezone`,
|
||||
`time_with_timezone`,
|
||||
`time_without_timezone`,
|
||||
`date`,
|
||||
parse_json(`unknown`)
|
||||
${'$'}{cdc_deleted_at_name}
|
||||
from unnest([
|
||||
STRUCT<
|
||||
_airbyte_raw_id STRING,
|
||||
_airbyte_extracted_at TIMESTAMP,
|
||||
_airbyte_meta STRING,
|
||||
_airbyte_generation_id INTEGER,
|
||||
`id1` STRING,
|
||||
`id2` STRING,
|
||||
`updated_at` TIMESTAMP,
|
||||
`struct` STRING,
|
||||
`array` STRING,
|
||||
`string` STRING,
|
||||
`number` STRING,
|
||||
`integer` STRING,
|
||||
`boolean` STRING,
|
||||
`timestamp_with_timezone` TIMESTAMP,
|
||||
`timestamp_without_timezone` DATETIME,
|
||||
`time_with_timezone` STRING,
|
||||
`time_without_timezone` TIME,
|
||||
`date` DATE,
|
||||
`unknown` STRING
|
||||
${'$'}{cdc_deleted_at_decl}
|
||||
>
|
||||
${'$'}{records}
|
||||
])
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
private fun stringifyRecords(records: List<JsonNode>, columnNames: List<String>): String {
|
||||
return records
|
||||
.stream() // For each record, convert it to a string like "(rawId, extractedAt,
|
||||
// loadedAt, data)"
|
||||
.map { record: JsonNode ->
|
||||
columnNames
|
||||
.stream()
|
||||
.map { fieldName: String? -> record[fieldName] }
|
||||
.map { r: JsonNode? ->
|
||||
if (r == null) {
|
||||
return@map "NULL"
|
||||
}
|
||||
val stringContents =
|
||||
if (r.isTextual) {
|
||||
r.asText()
|
||||
} else {
|
||||
r.toString()
|
||||
}
|
||||
'"'.toString() +
|
||||
stringContents // Serialized json might contain backslashes and double
|
||||
// quotes. Escape them.
|
||||
.replace("\\", "\\\\")
|
||||
.replace("\"", "\\\"") +
|
||||
'"'
|
||||
}
|
||||
.collect(Collectors.joining(","))
|
||||
}
|
||||
.map { row: String -> "($row)" }
|
||||
.collect(Collectors.joining(","))
|
||||
}
|
||||
|
||||
@Throws(InterruptedException::class)
|
||||
override fun insertRawTableRecords(streamId: StreamId, records: List<JsonNode>) {
|
||||
val recordsText =
|
||||
stringifyRecords(records, JavaBaseConstants.V2_RAW_TABLE_COLUMN_NAMES_WITH_GENERATION)
|
||||
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
Map.of(
|
||||
"raw_table_id",
|
||||
streamId.rawTableId(BigQuerySqlGenerator.QUOTE),
|
||||
"records",
|
||||
recordsText
|
||||
)
|
||||
)
|
||||
.replace( // TODO: Perform a normal insert - edward
|
||||
"""
|
||||
INSERT INTO ${'$'}{raw_table_id} (_airbyte_raw_id, _airbyte_extracted_at, _airbyte_loaded_at, _airbyte_data, _airbyte_meta, _airbyte_generation_id)
|
||||
SELECT _airbyte_raw_id, _airbyte_extracted_at, _airbyte_loaded_at, _airbyte_data, _airbyte_meta, cast(_airbyte_generation_id as int64) FROM UNNEST([
|
||||
STRUCT<`_airbyte_raw_id` STRING, `_airbyte_extracted_at` TIMESTAMP, `_airbyte_loaded_at` TIMESTAMP, _airbyte_data STRING, _airbyte_meta STRING, `_airbyte_generation_id` STRING>
|
||||
${'$'}{records}
|
||||
])
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun insertV1RawTableRecords(streamId: StreamId, records: List<JsonNode>) {
|
||||
val recordsText = stringifyRecords(records, JavaBaseConstants.LEGACY_RAW_TABLE_COLUMNS)
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.newBuilder(
|
||||
StringSubstitutor(
|
||||
Map.of(
|
||||
"v1_raw_table_id",
|
||||
streamId.rawTableId(BigQuerySqlGenerator.QUOTE),
|
||||
"records",
|
||||
recordsText
|
||||
)
|
||||
)
|
||||
.replace(
|
||||
"""
|
||||
INSERT INTO ${'$'}{v1_raw_table_id} (_airbyte_ab_id, _airbyte_data, _airbyte_emitted_at)
|
||||
SELECT _airbyte_ab_id, _airbyte_data, _airbyte_emitted_at FROM UNNEST([
|
||||
STRUCT<`_airbyte_ab_id` STRING, _airbyte_data STRING, `_airbyte_emitted_at` TIMESTAMP>
|
||||
${'$'}{records}
|
||||
])
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
.build()
|
||||
)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
override fun dumpRawTableRecords(streamId: StreamId): List<JsonNode> {
|
||||
val result =
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT * FROM " + streamId.rawTableId(BigQuerySqlGenerator.QUOTE)
|
||||
)
|
||||
)
|
||||
return toJsonRecords(result)
|
||||
}
|
||||
|
||||
@Throws(Exception::class)
|
||||
@SuppressFBWarnings("NP_PARAMETER_MUST_BE_NONNULL_BUT_MARKED_AS_NULLABLE")
|
||||
override fun dumpFinalTableRecords(streamId: StreamId, suffix: String?): List<JsonNode> {
|
||||
val result =
|
||||
bq!!.query(
|
||||
QueryJobConfiguration.of(
|
||||
"SELECT * FROM " + streamId.finalTableId(BigQuerySqlGenerator.QUOTE, suffix!!)
|
||||
)
|
||||
)
|
||||
return toJsonRecords(result)
|
||||
}
|
||||
|
||||
override fun teardownNamespace(namespace: String) {
|
||||
bq!!.delete(namespace, BigQuery.DatasetDeleteOption.deleteContents())
|
||||
}
|
||||
|
||||
override val supportsSafeCast: Boolean
|
||||
get() = true
|
||||
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
override fun testCreateTableIncremental() {
|
||||
destinationHandler.execute(generator.createTable(incrementalDedupStream, "", false))
|
||||
|
||||
val table = bq!!.getTable(namespace, "users_final")
|
||||
// The table should exist
|
||||
Assertions.assertNotNull(table)
|
||||
val schema = table.getDefinition<TableDefinition>().schema
|
||||
// And we should know exactly what columns it contains
|
||||
Assertions
|
||||
.assertEquals( // Would be nice to assert directly against StandardSQLTypeName, but
|
||||
// bigquery returns schemas of
|
||||
// LegacySQLTypeName. So we have to translate.
|
||||
Schema.of(
|
||||
Field.newBuilder(
|
||||
"_airbyte_raw_id",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING)
|
||||
)
|
||||
.setMode(Field.Mode.REQUIRED)
|
||||
.build(),
|
||||
Field.newBuilder(
|
||||
"_airbyte_extracted_at",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)
|
||||
)
|
||||
.setMode(Field.Mode.REQUIRED)
|
||||
.build(),
|
||||
Field.newBuilder(
|
||||
"_airbyte_meta",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON)
|
||||
)
|
||||
.setMode(Field.Mode.REQUIRED)
|
||||
.build(),
|
||||
Field.newBuilder(
|
||||
"_airbyte_generation_id",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)
|
||||
)
|
||||
.build(),
|
||||
Field.of("id1", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)),
|
||||
Field.of("id2", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)),
|
||||
Field.of(
|
||||
"updated_at",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)
|
||||
),
|
||||
Field.of(
|
||||
"struct",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON)
|
||||
),
|
||||
Field.of(
|
||||
"array",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON)
|
||||
),
|
||||
Field.of(
|
||||
"string",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING)
|
||||
),
|
||||
Field.of(
|
||||
"number",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.NUMERIC)
|
||||
),
|
||||
Field.of(
|
||||
"integer",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.INT64)
|
||||
),
|
||||
Field.of(
|
||||
"boolean",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.BOOL)
|
||||
),
|
||||
Field.of(
|
||||
"timestamp_with_timezone",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIMESTAMP)
|
||||
),
|
||||
Field.of(
|
||||
"timestamp_without_timezone",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.DATETIME)
|
||||
),
|
||||
Field.of(
|
||||
"time_with_timezone",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.STRING)
|
||||
),
|
||||
Field.of(
|
||||
"time_without_timezone",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.TIME)
|
||||
),
|
||||
Field.of("date", LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.DATE)),
|
||||
Field.of(
|
||||
"unknown",
|
||||
LegacySQLTypeName.legacySQLTypeName(StandardSQLTypeName.JSON)
|
||||
)
|
||||
),
|
||||
schema
|
||||
)
|
||||
// TODO this should assert partitioning/clustering configs
|
||||
}
|
||||
|
||||
@Test
|
||||
@Throws(InterruptedException::class)
|
||||
fun testCreateTableInOtherRegion() {
|
||||
val destinationHandler = BigQueryDestinationHandler(bq!!, "asia-east1")
|
||||
// We're creating the dataset in the wrong location in the @BeforeEach block. Explicitly
|
||||
// delete it.
|
||||
bq!!.getDataset(namespace).delete()
|
||||
val sqlGenerator = BigQuerySqlGenerator(projectId, "asia-east1")
|
||||
destinationHandler.execute(sqlGenerator.createSchema(namespace))
|
||||
destinationHandler.execute(sqlGenerator.createTable(incrementalDedupStream, "", false))
|
||||
|
||||
// Empirically, it sometimes takes Bigquery nearly 30 seconds to propagate the dataset's
|
||||
// existence.
|
||||
// Give ourselves 2 minutes just in case.
|
||||
for (i in 0..119) {
|
||||
val dataset = bq!!.getDataset(DatasetId.of(bq!!.options.projectId, namespace))
|
||||
if (dataset == null) {
|
||||
LOGGER.info("Sleeping and trying again... ({})", i)
|
||||
Thread.sleep(1000)
|
||||
} else {
|
||||
Assertions.assertEquals("asia-east1", dataset.location)
|
||||
return
|
||||
}
|
||||
}
|
||||
Assertions.fail<Any>("Dataset does not exist")
|
||||
}
|
||||
|
||||
/**
|
||||
* Bigquery column names aren't allowed to start with certain prefixes. Verify that we throw an
|
||||
* error in these cases.
|
||||
*/
|
||||
@ParameterizedTest
|
||||
@ValueSource(
|
||||
strings =
|
||||
["_table_", "_file_", "_partition_", "_row_timestamp_", "__root__", "_colidentifier_"]
|
||||
)
|
||||
fun testFailureOnReservedColumnNamePrefix(prefix: String) {
|
||||
val columns = java.util.LinkedHashMap<ColumnId, AirbyteType>()
|
||||
columns[generator.buildColumnId(prefix + "the_column_name")] = AirbyteProtocolType.STRING
|
||||
val stream =
|
||||
StreamConfig(
|
||||
streamId,
|
||||
ImportType.APPEND,
|
||||
emptyList(),
|
||||
Optional.empty(),
|
||||
columns,
|
||||
0,
|
||||
0,
|
||||
0
|
||||
)
|
||||
|
||||
val createTable = generator.createTable(stream, "", false)
|
||||
Assertions.assertThrows(BigQueryException::class.java) {
|
||||
destinationHandler.execute(createTable)
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Something about this test is borked on bigquery. It fails because the raw table doesn't
|
||||
* exist, but you can go into the UI and see that it does exist.
|
||||
*/
|
||||
@Disabled
|
||||
@Throws(Exception::class)
|
||||
override fun noCrashOnSpecialCharacters(specialChars: String) {
|
||||
super.noCrashOnSpecialCharacters(specialChars)
|
||||
}
|
||||
|
||||
/**
|
||||
* Bigquery doesn't handle frequent INSERT/DELETE statements on a single table very well. So we
|
||||
* don't have real state handling. Disable this test.
|
||||
*/
|
||||
@Disabled
|
||||
@Test
|
||||
@Throws(Exception::class)
|
||||
override fun testStateHandling() {
|
||||
super.testStateHandling()
|
||||
}
|
||||
|
||||
@Disabled
|
||||
override fun testLongIdentifierHandling() {
|
||||
super.testLongIdentifierHandling()
|
||||
}
|
||||
|
||||
companion object {
|
||||
private val LOGGER: Logger =
|
||||
LoggerFactory.getLogger(BigQuerySqlGeneratorIntegrationTest::class.java)
|
||||
|
||||
private var bq: BigQuery? = null
|
||||
private var projectId: String? = null
|
||||
private var datasetLocation: String? = null
|
||||
|
||||
@BeforeAll
|
||||
@Throws(Exception::class)
|
||||
@JvmStatic
|
||||
fun setupBigquery() {
|
||||
val rawConfig = Files.readString(Path.of("secrets/credentials-gcs-staging.json"))
|
||||
val config = deserialize(rawConfig)
|
||||
bq = getBigQuery(config)
|
||||
|
||||
projectId = config[BigQueryConsts.CONFIG_PROJECT_ID].asText()
|
||||
datasetLocation = config[BigQueryConsts.CONFIG_DATASET_LOCATION].asText()
|
||||
}
|
||||
|
||||
/**
|
||||
* TableResult contains records in a somewhat nonintuitive format (and it avoids loading
|
||||
* them all into memory). That's annoying for us since we're working with small test data,
|
||||
* so just pull everything into a list.
|
||||
*/
|
||||
fun toJsonRecords(result: TableResult): List<JsonNode> {
|
||||
return result
|
||||
.streamAll()
|
||||
.map { row: FieldValueList -> toJson(result.schema!!, row) }
|
||||
.toList()
|
||||
}
|
||||
|
||||
/**
|
||||
* FieldValueList stores everything internally as string (I think?) but provides conversions
|
||||
* to more useful types. This method does that conversion, using the schema to determine
|
||||
* which type is most appropriate. Then we just dump everything into a jsonnode for interop
|
||||
* with RecordDiffer.
|
||||
*/
|
||||
private fun toJson(schema: Schema, row: FieldValueList): JsonNode {
|
||||
val json = emptyObject() as ObjectNode
|
||||
for (i in schema.fields.indices) {
|
||||
val field = schema.fields[i]
|
||||
val value = row[i]
|
||||
val typedValue: JsonNode
|
||||
if (!value.isNull) {
|
||||
typedValue =
|
||||
when (field.type.standardType) {
|
||||
StandardSQLTypeName.BOOL -> jsonNode(value.booleanValue)
|
||||
StandardSQLTypeName.INT64 -> jsonNode(value.longValue)
|
||||
StandardSQLTypeName.FLOAT64 -> jsonNode(value.doubleValue)
|
||||
StandardSQLTypeName.NUMERIC,
|
||||
StandardSQLTypeName.BIGNUMERIC -> jsonNode(value.numericValue)
|
||||
StandardSQLTypeName.STRING -> jsonNode(value.stringValue)
|
||||
StandardSQLTypeName.TIMESTAMP ->
|
||||
jsonNode(value.timestampInstant.toString())
|
||||
StandardSQLTypeName.DATE,
|
||||
StandardSQLTypeName.DATETIME,
|
||||
StandardSQLTypeName.TIME -> jsonNode(value.stringValue)
|
||||
StandardSQLTypeName.JSON ->
|
||||
jsonNode(deserializeExact(value.stringValue))
|
||||
else -> jsonNode(value.stringValue)
|
||||
}
|
||||
json.set<JsonNode>(field.name, typedValue)
|
||||
}
|
||||
}
|
||||
return json
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,32 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import org.junit.jupiter.api.Disabled
|
||||
|
||||
class BigQueryStandardInsertsRawOverrideDisableTypingDedupingTest :
|
||||
AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-disabletd-standard-raw-override.json"
|
||||
|
||||
override val rawDataset: String
|
||||
get() = "overridden_raw_dataset"
|
||||
|
||||
override fun disableFinalTableComparison(): Boolean {
|
||||
return true
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Throws(Exception::class)
|
||||
override fun testRemovingPKNonNullIndexes() {
|
||||
// Do nothing.
|
||||
}
|
||||
|
||||
@Disabled
|
||||
@Throws(Exception::class)
|
||||
override fun identicalNameSimultaneousSync() {
|
||||
// TODO: create fixtures to verify how raw tables are affected. Base tests check for final
|
||||
// tables.
|
||||
}
|
||||
}
|
||||
@@ -0,0 +1,12 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
class BigQueryStandardInsertsRawOverrideTypingDedupingTest : AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-standard-raw-override.json"
|
||||
|
||||
override val rawDataset: String
|
||||
get() = "overridden_raw_dataset"
|
||||
}
|
||||
@@ -0,0 +1,9 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
class BigQueryStandardInsertsTypingDedupingTest : AbstractBigQueryTypingDedupingTest() {
|
||||
override val configPath: String
|
||||
get() = "secrets/credentials-1s1t-standard.json"
|
||||
}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "old_cursor": 1, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
@@ -0,0 +1,4 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43}
|
||||
@@ -0,0 +1,5 @@
|
||||
// Keep the Alice record with more recent updated_at
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "name": "Someone completely different"}
|
||||
@@ -0,0 +1,6 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
// Invalid columns are nulled out (i.e. SQL null, not JSON null)
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
@@ -0,0 +1,6 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}}
|
||||
// Invalid data is still allowed in the raw table.
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
// Charlie wasn't reemitted with updated_at, so it still has a null cursor
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
@@ -0,0 +1,7 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 0, "_ab_cdc_deleted_at": null, "name" :"Alice", "address": {"city": "San Francisco", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "old_cursor": 1, "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "old_cursor": 2, "name": "Bob", "address": {"city": "Boston", "state": "MA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 2, "id2": 200, "old_cursor": 3, "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}}
|
||||
@@ -0,0 +1,9 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}
|
||||
@@ -0,0 +1,9 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}
|
||||
@@ -0,0 +1,3 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
@@ -0,0 +1,4 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
// Delete Bob, keep Charlie
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00Z", "name": "Someone completely different v2"}
|
||||
@@ -0,0 +1,4 @@
|
||||
// Only sync2 messages present in overwrite mode
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
@@ -0,0 +1,10 @@
|
||||
// We keep the records from the first sync
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}}
|
||||
// And append the records from the second sync
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}}
|
||||
@@ -0,0 +1,2 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2001-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Someone completely different v2"}}
|
||||
@@ -0,0 +1,9 @@
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes":[]}, "_airbyte_generation_id": 0, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes":[]}, "_airbyte_generation_id": 0, "id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes":[]}, "_airbyte_generation_id": 0, "id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes":[]}, "_airbyte_generation_id": 0, "id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes":[]}, "_airbyte_generation_id": 0, "id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}
|
||||
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "name": "Bob", "address": {"city": "New York", "state": "NY"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes":[]}, "_airbyte_generation_id": 43, "id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}
|
||||
@@ -0,0 +1,11 @@
|
||||
// We keep the records from the first sync.
|
||||
// Note that sync ID and generation ID both default to 0.
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0, "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0, "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}}
|
||||
// And append the records from the second sync
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_meta": {"sync_id": 42, "changes": []}, "_airbyte_generation_id": 43, "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}}
|
||||
@@ -0,0 +1,10 @@
|
||||
// We keep the records from the first sync
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "San Francisco", "state": "CA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-01T00:01:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Los Angeles", "state": "CA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-01T00:02:00Z", "name": "Bob", "address": {"city": "Boston", "state": "MA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 2, "id2": 200, "updated_at": "2000-01-01T00:03:00Z", "name": "Charlie", "age": 42, "registration_date": "2023-12-23"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:01Z", "_airbyte_data": {"id1": 3, "id2": 200, "updated_at": "2000-01-01T00:04:00Z", "name": "a\bb\fc\nd\re\tf`~!@#$%^&*()_+-=[]\\{}|'\",./<>?"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 43}
|
||||
// And append the records from the second sync
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 200, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Alice", "address": {"city": "Seattle", "state": "WA"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:00:00Z", "_ab_cdc_deleted_at": null, "name": "Bob", "address": {"city": "New York", "state": "NY"}}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
{"_airbyte_extracted_at": "1970-01-01T00:00:02Z", "_airbyte_data": {"id1": 1, "id2": 201, "updated_at": "2000-01-02T00:01:00Z", "_ab_cdc_deleted_at": "1970-01-01T00:00:00Z"}, "_airbyte_meta": {"changes":[],"sync_id":42}, "_airbyte_generation_id": 44}
|
||||
@@ -1,170 +0,0 @@
|
||||
{
|
||||
"documentationUrl" : "https://docs.airbyte.com/integrations/destinations/bigquery",
|
||||
"connectionSpecification" : {
|
||||
"$schema" : "http://json-schema.org/draft-07/schema#",
|
||||
"title" : "Bigquery Specification",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"properties" : {
|
||||
"project_id" : {
|
||||
"type" : "string",
|
||||
"description" : "The GCP project ID for the project containing the target BigQuery dataset. Read more <a href=\"https://cloud.google.com/resource-manager/docs/creating-managing-projects#identifying_projects\">here</a>.",
|
||||
"title" : "Project ID",
|
||||
"group" : "connection",
|
||||
"order" : 0
|
||||
},
|
||||
"dataset_location" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "EU", "US", "africa-south1", "asia-east1", "asia-east2", "asia-northeast1", "asia-northeast2", "asia-northeast3", "asia-south1", "asia-south2", "asia-southeast1", "asia-southeast2", "australia-southeast1", "australia-southeast2", "europe-central2", "europe-north1", "europe-north2", "europe-southwest1", "europe-west1", "europe-west2", "europe-west3", "europe-west4", "europe-west6", "europe-west8", "europe-west9", "europe-west10", "europe-west12", "me-central1", "me-central2", "me-west1", "northamerica-northeast1", "northamerica-northeast2", "northamerica-south1", "southamerica-east1", "southamerica-west1", "us-central1", "us-east1", "us-east4", "us-east5", "us-south1", "us-west1", "us-west2", "us-west3", "us-west4" ],
|
||||
"description" : "The location of the dataset. Warning: Changes made after creation will not be applied. Read more <a href=\"https://cloud.google.com/bigquery/docs/locations\">here</a>.",
|
||||
"title" : "Dataset Location",
|
||||
"group" : "connection",
|
||||
"order" : 1
|
||||
},
|
||||
"dataset_id" : {
|
||||
"type" : "string",
|
||||
"description" : "The default BigQuery Dataset ID that tables are replicated to if the source does not specify a namespace. Read more <a href=\"https://cloud.google.com/bigquery/docs/datasets#create-dataset\">here</a>.",
|
||||
"title" : "Default Dataset ID",
|
||||
"group" : "connection",
|
||||
"order" : 2
|
||||
},
|
||||
"loading_method" : {
|
||||
"oneOf" : [ {
|
||||
"title" : "Batched Standard Inserts",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"description" : "Direct loading using batched SQL INSERT statements. This method uses the BigQuery driver to convert large INSERT statements into file uploads automatically.",
|
||||
"properties" : {
|
||||
"method" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "Standard" ],
|
||||
"default" : "Standard"
|
||||
}
|
||||
},
|
||||
"required" : [ "method" ]
|
||||
}, {
|
||||
"title" : "GCS Staging",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"description" : "Writes large batches of records to a file, uploads the file to GCS, then uses COPY INTO to load your data into BigQuery.",
|
||||
"properties" : {
|
||||
"method" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "GCS Staging" ],
|
||||
"default" : "GCS Staging"
|
||||
},
|
||||
"credential" : {
|
||||
"oneOf" : [ {
|
||||
"title" : "HMAC key",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"properties" : {
|
||||
"credential_type" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "HMAC_KEY" ],
|
||||
"default" : "HMAC_KEY"
|
||||
},
|
||||
"hmac_key_access_id" : {
|
||||
"type" : "string",
|
||||
"description" : "HMAC key access ID. When linked to a service account, this ID is 61 characters long; when linked to a user account, it is 24 characters long.",
|
||||
"title" : "HMAC Access Key",
|
||||
"examples" : [ "1234567890abcdefghij1234" ],
|
||||
"airbyte_secret" : true,
|
||||
"order" : 0
|
||||
},
|
||||
"hmac_key_secret" : {
|
||||
"type" : "string",
|
||||
"description" : "The corresponding secret for the access ID. It is a 40-character base-64 encoded string.",
|
||||
"title" : "HMAC Secret",
|
||||
"examples" : [ "1234567890abcdefghij1234567890ABCDEFGHIJ" ],
|
||||
"airbyte_secret" : true,
|
||||
"order" : 1
|
||||
}
|
||||
},
|
||||
"required" : [ "credential_type", "hmac_key_access_id", "hmac_key_secret" ]
|
||||
} ],
|
||||
"description" : "An HMAC key is a type of credential and can be associated with a service account or a user account in Cloud Storage. Read more <a href=\"https://cloud.google.com/storage/docs/authentication/hmackeys\">here</a>.",
|
||||
"title" : "Credential",
|
||||
"order" : 0,
|
||||
"type" : "object"
|
||||
},
|
||||
"keep_files_in_gcs-bucket" : {
|
||||
"type" : "string",
|
||||
"default" : "Delete all tmp files from GCS",
|
||||
"enum" : [ "Delete all tmp files from GCS", "Keep all tmp files in GCS" ],
|
||||
"description" : "This upload method is supposed to temporary store records in GCS bucket. By this select you can chose if these records should be removed from GCS when migration has finished. The default \"Delete all tmp files from GCS\" value is used if not set explicitly.",
|
||||
"title" : "GCS Tmp Files Post-Processing",
|
||||
"order" : 3
|
||||
},
|
||||
"gcs_bucket_name" : {
|
||||
"type" : "string",
|
||||
"description" : "The name of the GCS bucket. Read more <a href=\"https://cloud.google.com/storage/docs/naming-buckets\">here</a>.",
|
||||
"title" : "GCS Bucket Name",
|
||||
"examples" : [ "airbyte_sync" ],
|
||||
"order" : 1
|
||||
},
|
||||
"gcs_bucket_path" : {
|
||||
"type" : "string",
|
||||
"description" : "Directory under the GCS bucket where data will be written.",
|
||||
"title" : "GCS Bucket Path",
|
||||
"examples" : [ "data_sync/test" ],
|
||||
"order" : 2
|
||||
}
|
||||
},
|
||||
"required" : [ "method", "credential", "gcs_bucket_name", "gcs_bucket_path" ]
|
||||
} ],
|
||||
"description" : "The way data will be uploaded to BigQuery.",
|
||||
"title" : "Loading Method",
|
||||
"group" : "connection",
|
||||
"order" : 3,
|
||||
"display_type" : "radio",
|
||||
"type" : "object"
|
||||
},
|
||||
"credentials_json" : {
|
||||
"type" : "string",
|
||||
"description" : "The contents of the JSON service account key. Check out the <a href=\"https://docs.airbyte.com/integrations/destinations/bigquery#service-account-key\">docs</a> if you need help generating this key. Default credentials will be used if this field is left empty.",
|
||||
"title" : "Service Account Key JSON (Required for cloud, optional for open-source)",
|
||||
"group" : "connection",
|
||||
"order" : 4,
|
||||
"airbyte_secret" : true,
|
||||
"always_show" : true
|
||||
},
|
||||
"transformation_priority" : {
|
||||
"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
|
||||
},
|
||||
"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)",
|
||||
"group" : "advanced",
|
||||
"order" : 8,
|
||||
"default" : false
|
||||
}
|
||||
},
|
||||
"required" : [ "project_id", "dataset_location", "dataset_id" ],
|
||||
"groups" : [ {
|
||||
"id" : "connection",
|
||||
"title" : "Connection"
|
||||
}, {
|
||||
"id" : "advanced",
|
||||
"title" : "Advanced"
|
||||
} ]
|
||||
},
|
||||
"supportsIncremental" : true,
|
||||
"supportsNormalization" : false,
|
||||
"supportsDBT" : false,
|
||||
"supported_destination_sync_modes" : [ "overwrite", "append", "append_dedup" ]
|
||||
}
|
||||
@@ -1,170 +0,0 @@
|
||||
{
|
||||
"documentationUrl" : "https://docs.airbyte.com/integrations/destinations/bigquery",
|
||||
"connectionSpecification" : {
|
||||
"$schema" : "http://json-schema.org/draft-07/schema#",
|
||||
"title" : "Bigquery Specification",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"properties" : {
|
||||
"project_id" : {
|
||||
"type" : "string",
|
||||
"description" : "The GCP project ID for the project containing the target BigQuery dataset. Read more <a href=\"https://cloud.google.com/resource-manager/docs/creating-managing-projects#identifying_projects\">here</a>.",
|
||||
"title" : "Project ID",
|
||||
"group" : "connection",
|
||||
"order" : 0
|
||||
},
|
||||
"dataset_location" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "EU", "US", "africa-south1", "asia-east1", "asia-east2", "asia-northeast1", "asia-northeast2", "asia-northeast3", "asia-south1", "asia-south2", "asia-southeast1", "asia-southeast2", "australia-southeast1", "australia-southeast2", "europe-central2", "europe-north1", "europe-north2", "europe-southwest1", "europe-west1", "europe-west2", "europe-west3", "europe-west4", "europe-west6", "europe-west8", "europe-west9", "europe-west10", "europe-west12", "me-central1", "me-central2", "me-west1", "northamerica-northeast1", "northamerica-northeast2", "northamerica-south1", "southamerica-east1", "southamerica-west1", "us-central1", "us-east1", "us-east4", "us-east5", "us-south1", "us-west1", "us-west2", "us-west3", "us-west4" ],
|
||||
"description" : "The location of the dataset. Warning: Changes made after creation will not be applied. Read more <a href=\"https://cloud.google.com/bigquery/docs/locations\">here</a>.",
|
||||
"title" : "Dataset Location",
|
||||
"group" : "connection",
|
||||
"order" : 1
|
||||
},
|
||||
"dataset_id" : {
|
||||
"type" : "string",
|
||||
"description" : "The default BigQuery Dataset ID that tables are replicated to if the source does not specify a namespace. Read more <a href=\"https://cloud.google.com/bigquery/docs/datasets#create-dataset\">here</a>.",
|
||||
"title" : "Default Dataset ID",
|
||||
"group" : "connection",
|
||||
"order" : 2
|
||||
},
|
||||
"loading_method" : {
|
||||
"oneOf" : [ {
|
||||
"title" : "Batched Standard Inserts",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"description" : "Direct loading using batched SQL INSERT statements. This method uses the BigQuery driver to convert large INSERT statements into file uploads automatically.",
|
||||
"properties" : {
|
||||
"method" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "Standard" ],
|
||||
"default" : "Standard"
|
||||
}
|
||||
},
|
||||
"required" : [ "method" ]
|
||||
}, {
|
||||
"title" : "GCS Staging",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"description" : "Writes large batches of records to a file, uploads the file to GCS, then uses COPY INTO to load your data into BigQuery.",
|
||||
"properties" : {
|
||||
"method" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "GCS Staging" ],
|
||||
"default" : "GCS Staging"
|
||||
},
|
||||
"credential" : {
|
||||
"oneOf" : [ {
|
||||
"title" : "HMAC key",
|
||||
"type" : "object",
|
||||
"additionalProperties" : true,
|
||||
"properties" : {
|
||||
"credential_type" : {
|
||||
"type" : "string",
|
||||
"enum" : [ "HMAC_KEY" ],
|
||||
"default" : "HMAC_KEY"
|
||||
},
|
||||
"hmac_key_access_id" : {
|
||||
"type" : "string",
|
||||
"description" : "HMAC key access ID. When linked to a service account, this ID is 61 characters long; when linked to a user account, it is 24 characters long.",
|
||||
"title" : "HMAC Access Key",
|
||||
"examples" : [ "1234567890abcdefghij1234" ],
|
||||
"airbyte_secret" : true,
|
||||
"order" : 0
|
||||
},
|
||||
"hmac_key_secret" : {
|
||||
"type" : "string",
|
||||
"description" : "The corresponding secret for the access ID. It is a 40-character base-64 encoded string.",
|
||||
"title" : "HMAC Secret",
|
||||
"examples" : [ "1234567890abcdefghij1234567890ABCDEFGHIJ" ],
|
||||
"airbyte_secret" : true,
|
||||
"order" : 1
|
||||
}
|
||||
},
|
||||
"required" : [ "credential_type", "hmac_key_access_id", "hmac_key_secret" ]
|
||||
} ],
|
||||
"description" : "An HMAC key is a type of credential and can be associated with a service account or a user account in Cloud Storage. Read more <a href=\"https://cloud.google.com/storage/docs/authentication/hmackeys\">here</a>.",
|
||||
"title" : "Credential",
|
||||
"order" : 0,
|
||||
"type" : "object"
|
||||
},
|
||||
"keep_files_in_gcs-bucket" : {
|
||||
"type" : "string",
|
||||
"default" : "Delete all tmp files from GCS",
|
||||
"enum" : [ "Delete all tmp files from GCS", "Keep all tmp files in GCS" ],
|
||||
"description" : "This upload method is supposed to temporary store records in GCS bucket. By this select you can chose if these records should be removed from GCS when migration has finished. The default \"Delete all tmp files from GCS\" value is used if not set explicitly.",
|
||||
"title" : "GCS Tmp Files Post-Processing",
|
||||
"order" : 3
|
||||
},
|
||||
"gcs_bucket_name" : {
|
||||
"type" : "string",
|
||||
"description" : "The name of the GCS bucket. Read more <a href=\"https://cloud.google.com/storage/docs/naming-buckets\">here</a>.",
|
||||
"title" : "GCS Bucket Name",
|
||||
"examples" : [ "airbyte_sync" ],
|
||||
"order" : 1
|
||||
},
|
||||
"gcs_bucket_path" : {
|
||||
"type" : "string",
|
||||
"description" : "Directory under the GCS bucket where data will be written.",
|
||||
"title" : "GCS Bucket Path",
|
||||
"examples" : [ "data_sync/test" ],
|
||||
"order" : 2
|
||||
}
|
||||
},
|
||||
"required" : [ "method", "credential", "gcs_bucket_name", "gcs_bucket_path" ]
|
||||
} ],
|
||||
"description" : "The way data will be uploaded to BigQuery.",
|
||||
"title" : "Loading Method",
|
||||
"group" : "connection",
|
||||
"order" : 3,
|
||||
"display_type" : "radio",
|
||||
"type" : "object"
|
||||
},
|
||||
"credentials_json" : {
|
||||
"type" : "string",
|
||||
"description" : "The contents of the JSON service account key. Check out the <a href=\"https://docs.airbyte.com/integrations/destinations/bigquery#service-account-key\">docs</a> if you need help generating this key. Default credentials will be used if this field is left empty.",
|
||||
"title" : "Service Account Key JSON (Required for cloud, optional for open-source)",
|
||||
"group" : "connection",
|
||||
"order" : 4,
|
||||
"airbyte_secret" : true,
|
||||
"always_show" : true
|
||||
},
|
||||
"transformation_priority" : {
|
||||
"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
|
||||
},
|
||||
"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)",
|
||||
"group" : "advanced",
|
||||
"order" : 8,
|
||||
"default" : false
|
||||
}
|
||||
},
|
||||
"required" : [ "project_id", "dataset_location", "dataset_id" ],
|
||||
"groups" : [ {
|
||||
"id" : "connection",
|
||||
"title" : "Connection"
|
||||
}, {
|
||||
"id" : "advanced",
|
||||
"title" : "Advanced"
|
||||
} ]
|
||||
},
|
||||
"supportsIncremental" : true,
|
||||
"supportsNormalization" : false,
|
||||
"supportsDBT" : false,
|
||||
"supported_destination_sync_modes" : [ "overwrite", "append", "append_dedup" ]
|
||||
}
|
||||
@@ -0,0 +1,7 @@
|
||||
{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "_airbyte_generation_id": 42}
|
||||
{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "unknown": null, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "_airbyte_generation_id": 42}
|
||||
{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 42, "changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}, "_airbyte_generation_id": 42}
|
||||
{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "unknown": null, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": [{"field": "struct", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "array", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "number", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "integer", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "boolean", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "timestamp_with_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "timestamp_without_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "time_with_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "time_without_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "date", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}, "_airbyte_generation_id": 42}
|
||||
// Note that for numbers where we parse the value to JSON (struct, array, unknown) we lose precision.
|
||||
// But for numbers where we create a NUMBER column, we do not lose precision (see the `number` column).
|
||||
{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.17411800000001}, "array": [67.17411800000001], "unknown": 67.17411800000001, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "_airbyte_generation_id": 42}
|
||||
@@ -0,0 +1,5 @@
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}, "_airbyte_generation_id": 42}
|
||||
{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}, "_airbyte_generation_id": 42}
|
||||
{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}, "_airbyte_meta": {"sync_id": 42, "changes": [{"field": "string", "change": "NULLED", "reason": "SOURCE_SERIALIZATION_ERROR"}]}, "_airbyte_generation_id": 42}
|
||||
{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}, "_airbyte_generation_id": 42}
|
||||
{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}, "_airbyte_generation_id": 42}
|
||||
@@ -0,0 +1,7 @@
|
||||
{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "unknown": null, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "unknown": null, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 0, "changes": [{"field": "struct", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "array", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "number", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "integer", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "boolean", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "timestamp_with_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "timestamp_without_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "time_with_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "time_without_timezone", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}, {"field": "date", "change":"NULLED","reason":"DESTINATION_TYPECAST_ERROR"}]}, "_airbyte_generation_id": 0}
|
||||
// Note that for numbers where we parse the value to JSON (struct, array, unknown) we lose precision.
|
||||
// But for numbers where we create a NUMBER column, we do not lose precision (see the `number` column).
|
||||
{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.17411800000001}, "array": [67.17411800000001], "unknown": 67.17411800000001, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
@@ -0,0 +1,5 @@
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "foo", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}, "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": null, "struct": null, "string": null, "number": null, "integer": null, "boolean": null, "timestamp_with_timezone": null, "timestamp_without_timezone": null, "time_with_timezone": null, "time_without_timezone": null, "date": null, "unknown": null}, "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z"}, "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": {}, "struct": [], "string": null, "number": "foo", "integer": "bar", "boolean": "fizz", "timestamp_with_timezone": {}, "timestamp_without_timezone": {}, "time_with_timezone": {}, "time_without_timezone": {}, "date": "airbyte", "unknown": null}, "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "number": 67.174118, "struct": {"nested_number": 67.174118}, "array": [67.174118], "unknown": 67.174118}, "_airbyte_meta": {"sync_id": 0, "changes": []}, "_airbyte_generation_id": 0}
|
||||
@@ -0,0 +1,2 @@
|
||||
{"_airbyte_raw_id": "80c99b54-54b4-43bd-b51b-1f67dafa2c52", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}
|
||||
{"_airbyte_raw_id": "b9ac9f01-abc1-4e7c-89e5-eac9223d5726", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": [{"change":"NULLED","field":"integer","reason":"DESTINATION_TYPECAST_ERROR"}]}, "id1": 2, "id2": 100, "updated_at": "2023-01-01T03:00:01Z", "string": "Bob"}
|
||||
@@ -0,0 +1,4 @@
|
||||
{"_airbyte_raw_id": "d7b81af0-01da-4846-a650-cc398986bc99", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "string": "Alice", "struct": {"city": "San Francisco", "state": "CA"}, "integer": 42}}
|
||||
{"_airbyte_raw_id": "80c99b54-54b4-43bd-b51b-1f67dafa2c52", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00Z", "string": "Alice", "struct": {"city": "San Diego", "state": "CA"}, "integer": 84}}
|
||||
{"_airbyte_raw_id": "ad690bfb-c2c2-4172-bd73-a16c86ccbb67", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T03:00:00Z", "string": "Bob", "integer": 126}}
|
||||
{"_airbyte_raw_id": "b9ac9f01-abc1-4e7c-89e5-eac9223d5726", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T03:00:01Z", "string": "Bob", "integer": "oops"}}
|
||||
@@ -0,0 +1,5 @@
|
||||
{"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "[\"I\",\"am\",\"an\",\"array\"]", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
{"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "{\"I\":\"am\",\"an\":\"object\"}", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
{"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "true", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
{"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "3.14", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
{"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "I am a valid json string", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}, "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
@@ -0,0 +1,5 @@
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": ["I", "am", "an", "array"], "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}}
|
||||
{"_airbyte_raw_id": "53ce75a5-5bcc-47a3-b45c-96c2015cfe35", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 2, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": {"I": "am", "an": "object"}, "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}}
|
||||
{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fbe", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 3, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": true, "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}}
|
||||
{"_airbyte_raw_id": "84242b60-3a34-4531-ad75-a26702960a9a", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 4, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": 3.14, "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}}
|
||||
{"_airbyte_raw_id": "a4a783b5-7729-4d0b-b659-48ceb08713f1", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 5, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "array": ["foo"], "struct": {"foo": "bar"}, "string": "I am a valid json string", "number": 42.1, "integer": 42, "boolean": true, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "timestamp_without_timezone": "2023-01-23T12:34:56", "time_with_timezone": "12:34:56Z", "time_without_timezone": "12:34:56", "date": "2023-01-23", "unknown": {}}}
|
||||
@@ -0,0 +1 @@
|
||||
{"id1": 6, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "IamACaseSensitiveColumnName": "Case senstive value", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_raw_id": "7e1fac0c-017e-4ad6-bc78-334a34d64fce", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 6, "id2": 100, "updated_at": "2023-01-01T01:00:00Z", "IamACaseSensitiveColumnName": "Case senstive value"}}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {}}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_raw_id":"b2e0efc4-38a8-47ba-970c-8103f09f08d5","_airbyte_extracted_at":"2023-01-01T00:00:00Z","_airbyte_meta":{"changes":[]}, "current_date": "foo", "join": "bar"}
|
||||
@@ -0,0 +1,14 @@
|
||||
// Bigquery converts timestamp_with_timezone to UTC
|
||||
// But we're using STRING for time_with_timezone, so that column isn't modified at all
|
||||
{"_airbyte_raw_id": "14ba7c7f-e398-4e69-ac22-28d578400dbc", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56Z", "time_with_timezone": "12:34:56Z"}
|
||||
{"_airbyte_raw_id": "05028c5f-7813-4e9c-bd4b-387d1f8ba435", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56Z", "time_with_timezone": "12:34:56-08:00"}
|
||||
{"_airbyte_raw_id": "95dfb0c6-6a67-4ba0-9935-643bebc90437", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56Z", "time_with_timezone": "12:34:56-0800"}
|
||||
{"_airbyte_raw_id": "f3d8abe2-bb0f-4caf-8ddc-0641df02f3a9", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T20:34:56Z", "time_with_timezone": "12:34:56-08"}
|
||||
{"_airbyte_raw_id": "a81ed40a-2a49-488d-9714-d53e8b052968", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56Z", "time_with_timezone": "12:34:56+08:00"}
|
||||
{"_airbyte_raw_id": "c07763a0-89e6-4cb7-b7d0-7a34a7c9918a", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56Z", "time_with_timezone": "12:34:56+0800"}
|
||||
{"_airbyte_raw_id": "358d3b52-50ab-4e06-9094-039386f9bf0d", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T04:34:56Z", "time_with_timezone": "12:34:56+08"}
|
||||
{"_airbyte_raw_id": "db8200ac-b2b9-4b95-a053-8a0343042751", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_with_timezone": "2023-01-23T12:34:56.123Z", "time_with_timezone": "12:34:56.123Z"}
|
||||
|
||||
{"_airbyte_raw_id": "10ce5d93-6923-4217-a46f-103833837038", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56", "time_without_timezone": "12:34:56", "date": "2023-01-23"}
|
||||
// Bigquery returns 6 decimal places if there are any decimal places... but not for timestamp_with_timezone
|
||||
{"_airbyte_raw_id": "a7a6e176-7464-4a0b-b55c-b4f936e8d5a1", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "timestamp_without_timezone": "2023-01-23T12:34:56.123000", "time_without_timezone": "12:34:56.123000"}
|
||||
@@ -0,0 +1,10 @@
|
||||
// column renamings:
|
||||
// * $starts_with_dollar_sign -> _starts_with_dollar_sign
|
||||
// * includes"doublequote -> includes_doublequote
|
||||
// * includes'singlequote -> includes_singlequote
|
||||
// * includes`backtick -> includes_backtick
|
||||
// * includes$$doubledollar -> includes__doubledollar
|
||||
// * includes.period -> includes_period
|
||||
// columns with issues:
|
||||
// * endswithbackslash\ -> nulled out, and no error in airbyte_meta. This actually extracts the value correctly if _airbyte_data is a JSON column, but JSON_VALUE seems to return null if the data is a string and the key contains a backslash.
|
||||
{"_airbyte_raw_id": "7e7330a1-42fb-41ec-a955-52f18bd61964", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_meta": {"changes": []}, "id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00Z", "_starts_with_dollar_sign": "foo", "includes_doublequote": "foo", "includes_singlequote": "foo", "includes_backtick": "foo", "includes_period": "foo", "includes__doubledollar": "foo"}
|
||||
@@ -0,0 +1 @@
|
||||
{"_airbyte_raw_id": "7e7330a1-42fb-41ec-a955-52f18bd61964", "_airbyte_extracted_at": "2023-01-01T00:00:00Z", "_airbyte_data": {"id1": 1, "id2": 100, "updated_at": "2023-01-01T02:00:00Z", "$starts_with_dollar_sign": "foo", "includes\"doublequote": "foo", "includes'singlequote": "foo", "includes`backtick": "foo", "includes.period": "foo", "includes$$doubledollar": "foo", "endswithbackslash\\": "foo"}}
|
||||
@@ -7,6 +7,14 @@ import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
internal class BigQuerySQLNameTransformerTest {
|
||||
@Test
|
||||
fun testGetIdentifier() {
|
||||
RAW_TO_NORMALIZED_IDENTIFIERS.forEach { (raw: String?, normalized: String?) ->
|
||||
Assertions.assertEquals(normalized, INSTANCE.getIdentifier(raw))
|
||||
Assertions.assertEquals(normalized, INSTANCE.convertStreamName(raw))
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testGetNamespace() {
|
||||
RAW_TO_NORMALIZED_NAMESPACES.forEach { (raw: String?, normalized: String?) ->
|
||||
@@ -16,6 +24,19 @@ internal class BigQuerySQLNameTransformerTest {
|
||||
|
||||
companion object {
|
||||
private val INSTANCE = BigQuerySQLNameTransformer()
|
||||
private val RAW_TO_NORMALIZED_IDENTIFIERS: Map<String, String> =
|
||||
java.util.Map.of(
|
||||
"name-space",
|
||||
"name_space",
|
||||
"spécial_character",
|
||||
"special_character",
|
||||
"99namespace",
|
||||
"_99namespace",
|
||||
"*_namespace",
|
||||
"__namespace",
|
||||
"_namespace",
|
||||
"_namespace"
|
||||
)
|
||||
|
||||
private val RAW_TO_NORMALIZED_NAMESPACES: Map<String, String> =
|
||||
java.util.Map.of(
|
||||
|
||||
@@ -4,8 +4,8 @@
|
||||
package io.airbyte.integrations.destination.bigquery
|
||||
|
||||
import com.google.common.collect.ImmutableMap
|
||||
import io.airbyte.commons.json.Jsons.jsonNode
|
||||
import io.airbyte.integrations.destination.bigquery.BigQueryUtils.getDatasetId
|
||||
import io.airbyte.protocol.models.Jsons.jsonNode
|
||||
import java.util.stream.Stream
|
||||
import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.BeforeEach
|
||||
|
||||
@@ -0,0 +1,73 @@
|
||||
/*
|
||||
* Copyright (c) 2023 Airbyte, Inc., all rights reserved.
|
||||
*/
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import io.airbyte.commons.json.Jsons.deserialize
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.*
|
||||
import io.airbyte.protocol.models.v0.AirbyteStream
|
||||
import io.airbyte.protocol.models.v0.ConfiguredAirbyteStream
|
||||
import io.airbyte.protocol.models.v0.DestinationSyncMode
|
||||
import io.airbyte.protocol.models.v0.SyncMode
|
||||
import java.util.*
|
||||
import org.junit.jupiter.api.Assertions
|
||||
import org.junit.jupiter.api.Test
|
||||
|
||||
class BigQuerySqlGeneratorTest {
|
||||
private val generator = BigQuerySqlGenerator("foo", "US")
|
||||
|
||||
@Test
|
||||
fun testBuildColumnId() {
|
||||
// Uninteresting names are unchanged
|
||||
Assertions.assertEquals(ColumnId("foo", "foo", "foo"), generator.buildColumnId("foo"))
|
||||
}
|
||||
|
||||
@Test
|
||||
fun columnCollision() {
|
||||
val parser = CatalogParser(generator, "default_ns")
|
||||
val columns = LinkedHashMap<ColumnId, AirbyteType>()
|
||||
columns[ColumnId("CURRENT_DATE", "CURRENT_DATE", "current_date")] =
|
||||
AirbyteProtocolType.STRING
|
||||
columns[ColumnId("current_date_1", "current_date", "current_date_1")] =
|
||||
AirbyteProtocolType.INTEGER
|
||||
Assertions.assertEquals(
|
||||
StreamConfig(
|
||||
StreamId("bar", "foo", "airbyte_internal", "bar_raw__stream_foo", "bar", "foo"),
|
||||
ImportType.APPEND,
|
||||
emptyList(),
|
||||
Optional.empty(),
|
||||
columns,
|
||||
1,
|
||||
1,
|
||||
2
|
||||
),
|
||||
parser.toStreamConfig(
|
||||
ConfiguredAirbyteStream()
|
||||
.withSyncMode(SyncMode.INCREMENTAL)
|
||||
.withDestinationSyncMode(DestinationSyncMode.APPEND)
|
||||
.withGenerationId(1L)
|
||||
.withMinimumGenerationId(1L)
|
||||
.withSyncId(2L)
|
||||
.withStream(
|
||||
AirbyteStream()
|
||||
.withName("foo")
|
||||
.withNamespace("bar")
|
||||
.withJsonSchema(
|
||||
deserialize(
|
||||
"""
|
||||
{
|
||||
"type": "object",
|
||||
"properties": {
|
||||
"CURRENT_DATE": {"type": "string"},
|
||||
"current_date": {"type": "integer"}
|
||||
}
|
||||
}
|
||||
|
||||
""".trimIndent()
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
@@ -4,195 +4,122 @@
|
||||
package io.airbyte.integrations.destination.bigquery.typing_deduping
|
||||
|
||||
import com.google.cloud.bigquery.Clustering
|
||||
import com.google.cloud.bigquery.Field
|
||||
import com.google.cloud.bigquery.FieldList
|
||||
import com.google.cloud.bigquery.StandardSQLTypeName
|
||||
import com.google.cloud.bigquery.StandardTableDefinition
|
||||
import com.google.cloud.bigquery.TimePartitioning
|
||||
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.ArrayType
|
||||
import io.airbyte.cdk.load.data.BooleanType
|
||||
import io.airbyte.cdk.load.data.FieldType
|
||||
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.ObjectTypeWithoutSchema
|
||||
import io.airbyte.cdk.load.data.UnionType
|
||||
import io.airbyte.cdk.load.orchestration.db.ColumnNameMapping
|
||||
import com.google.common.collect.ImmutableList
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.AirbyteProtocolType
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.AirbyteType
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.Array
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.ColumnId
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.ImportType
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.StreamConfig
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.Struct
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.Union
|
||||
import io.airbyte.integrations.base.destination.typing_deduping.UnsupportedOneOf
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.clusteringMatches
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.partitioningMatches
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.BigQueryDestinationHandler.Companion.schemaContainAllFinalTableV2AirbyteColumns
|
||||
import io.airbyte.integrations.destination.bigquery.typing_deduping.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 java.util.Optional
|
||||
import java.util.stream.Collectors
|
||||
import java.util.stream.Stream
|
||||
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 {
|
||||
@Test
|
||||
fun testToDialectType() {
|
||||
val s = ObjectType(linkedMapOf())
|
||||
val a = ArrayType(FieldType(BooleanType, nullable = true))
|
||||
val s = Struct(LinkedHashMap())
|
||||
val a = Array(AirbyteProtocolType.BOOLEAN)
|
||||
|
||||
Assertions.assertEquals(StandardSQLTypeName.INT64, toDialectType(IntegerType))
|
||||
Assertions.assertEquals(
|
||||
StandardSQLTypeName.INT64,
|
||||
toDialectType((AirbyteProtocolType.INTEGER as AirbyteType))
|
||||
)
|
||||
Assertions.assertEquals(StandardSQLTypeName.JSON, toDialectType(s))
|
||||
Assertions.assertEquals(StandardSQLTypeName.JSON, toDialectType(a))
|
||||
Assertions.assertEquals(
|
||||
StandardSQLTypeName.JSON,
|
||||
toDialectType(UnionType(emptySet(), isLegacyUnion = false))
|
||||
toDialectType(UnsupportedOneOf(ArrayList()))
|
||||
)
|
||||
|
||||
var u = UnionType(setOf(s), isLegacyUnion = true)
|
||||
var u = Union(ImmutableList.of(s))
|
||||
Assertions.assertEquals(StandardSQLTypeName.JSON, toDialectType(u))
|
||||
u = UnionType(setOf(a), isLegacyUnion = true)
|
||||
u = Union(ImmutableList.of(a))
|
||||
Assertions.assertEquals(StandardSQLTypeName.JSON, toDialectType(u))
|
||||
u = UnionType(setOf(BooleanType, NumberType), isLegacyUnion = true)
|
||||
u = Union(ImmutableList.of(AirbyteProtocolType.BOOLEAN, AirbyteProtocolType.NUMBER))
|
||||
Assertions.assertEquals(StandardSQLTypeName.NUMERIC, toDialectType(u))
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testColumnsMatch() {
|
||||
val stream =
|
||||
DestinationStream(
|
||||
"foo",
|
||||
"bar",
|
||||
Append,
|
||||
ObjectType(linkedMapOf("a1" to FieldType(IntegerType, nullable = true))),
|
||||
generationId = 0,
|
||||
minimumGenerationId = 0,
|
||||
syncId = 0,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
)
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("a1" to "a2"))
|
||||
val existingTable = Mockito.mock(StandardTableDefinition::class.java, RETURNS_DEEP_STUBS)
|
||||
Mockito.`when`(existingTable.schema!!.fields)
|
||||
.thenReturn(FieldList.of(Field.of("a2", StandardSQLTypeName.INT64)))
|
||||
val alterTableReport =
|
||||
BigqueryDatabaseInitialStatusGatherer(Mockito.mock())
|
||||
.buildAlterTableReport(stream, columnNameMapping, existingTable)
|
||||
Assertions.assertAll(
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToAdd) },
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToRemove) },
|
||||
{ Assertions.assertEquals(emptySet<String>(), alterTableReport.columnsToChangeType) },
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testColumnsNotMatch() {
|
||||
val stream =
|
||||
DestinationStream(
|
||||
"foo",
|
||||
"bar",
|
||||
Append,
|
||||
ObjectType(
|
||||
linkedMapOf(
|
||||
"a1" to FieldType(IntegerType, nullable = true),
|
||||
"c1" to FieldType(IntegerType, nullable = true),
|
||||
)
|
||||
),
|
||||
generationId = 0,
|
||||
minimumGenerationId = 0,
|
||||
syncId = 0,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
)
|
||||
val columnNameMapping = ColumnNameMapping(mapOf("a1" to "a2", "c1" to "c2"))
|
||||
val existingTable = Mockito.mock(StandardTableDefinition::class.java, RETURNS_DEEP_STUBS)
|
||||
Mockito.`when`(existingTable.schema!!.fields)
|
||||
.thenReturn(
|
||||
FieldList.of(
|
||||
listOf(
|
||||
Field.of("a2", StandardSQLTypeName.STRING),
|
||||
Field.of("b2", StandardSQLTypeName.INT64)
|
||||
)
|
||||
)
|
||||
)
|
||||
val alterTableReport =
|
||||
BigqueryDatabaseInitialStatusGatherer(Mockito.mock())
|
||||
.buildAlterTableReport(stream, columnNameMapping, existingTable)
|
||||
Assertions.assertAll(
|
||||
{ Assertions.assertEquals(setOf("c2"), alterTableReport.columnsToAdd) },
|
||||
{ Assertions.assertEquals(setOf("b2"), alterTableReport.columnsToRemove) },
|
||||
{ Assertions.assertEquals(setOf("a2"), alterTableReport.columnsToChangeType) },
|
||||
)
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testClusteringMatches() {
|
||||
var stream =
|
||||
DestinationStream(
|
||||
"foo",
|
||||
"bar",
|
||||
Dedupe(
|
||||
listOf(listOf("bar")),
|
||||
emptyList(),
|
||||
),
|
||||
ObjectTypeWithoutSchema,
|
||||
generationId = 0,
|
||||
minimumGenerationId = 0,
|
||||
syncId = 0,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
StreamConfig(
|
||||
Mockito.mock(),
|
||||
ImportType.DEDUPE,
|
||||
listOf(ColumnId("foo", "bar", "fizz")),
|
||||
Optional.empty(),
|
||||
LinkedHashMap(),
|
||||
0,
|
||||
0,
|
||||
0
|
||||
)
|
||||
var columnNameMapping = ColumnNameMapping(mapOf("bar" to "foo"))
|
||||
|
||||
// Clustering is null
|
||||
val existingTable = Mockito.mock(StandardTableDefinition::class.java)
|
||||
Mockito.`when`(existingTable.clustering).thenReturn(null)
|
||||
Assertions.assertFalse(clusteringMatches(stream, columnNameMapping, existingTable))
|
||||
Assertions.assertFalse(clusteringMatches(stream, existingTable))
|
||||
|
||||
// Clustering does not contain all fields
|
||||
Mockito.`when`(existingTable.clustering)
|
||||
.thenReturn(Clustering.newBuilder().setFields(listOf("_airbyte_extracted_at")).build())
|
||||
Assertions.assertFalse(clusteringMatches(stream, columnNameMapping, existingTable))
|
||||
Assertions.assertFalse(clusteringMatches(stream, existingTable))
|
||||
|
||||
// Clustering matches
|
||||
stream =
|
||||
DestinationStream(
|
||||
"foo",
|
||||
"bar",
|
||||
Append,
|
||||
ObjectTypeWithoutSchema,
|
||||
generationId = 0,
|
||||
minimumGenerationId = 0,
|
||||
syncId = 0,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
StreamConfig(
|
||||
Mockito.mock(),
|
||||
ImportType.APPEND,
|
||||
emptyList(),
|
||||
Optional.empty(),
|
||||
LinkedHashMap(),
|
||||
0,
|
||||
0,
|
||||
0
|
||||
)
|
||||
Assertions.assertTrue(clusteringMatches(stream, columnNameMapping, existingTable))
|
||||
Assertions.assertTrue(clusteringMatches(stream, existingTable))
|
||||
|
||||
// Clustering only the first 3 PK columns (See
|
||||
// https://github.com/airbytehq/oncall/issues/2565)
|
||||
val expectedStreamColumnNames = listOf("a", "b", "c")
|
||||
Mockito.`when`(existingTable.clustering)
|
||||
.thenReturn(
|
||||
Clustering.newBuilder()
|
||||
.setFields(listOf("a2", "b2", "c2", "_airbyte_extracted_at"))
|
||||
.setFields(
|
||||
Stream.concat(
|
||||
expectedStreamColumnNames.stream(),
|
||||
Stream.of("_airbyte_extracted_at")
|
||||
)
|
||||
.collect(Collectors.toList())
|
||||
)
|
||||
.build()
|
||||
)
|
||||
stream =
|
||||
DestinationStream(
|
||||
"foo",
|
||||
"bar",
|
||||
Dedupe(
|
||||
listOf(listOf("a1"), listOf("b1"), listOf("c1"), listOf("d1"), listOf("e1")),
|
||||
emptyList()
|
||||
),
|
||||
ObjectTypeWithoutSchema,
|
||||
generationId = 0,
|
||||
minimumGenerationId = 0,
|
||||
syncId = 0,
|
||||
namespaceMapper = NamespaceMapper()
|
||||
StreamConfig(
|
||||
Mockito.mock(),
|
||||
ImportType.DEDUPE,
|
||||
Stream.concat(expectedStreamColumnNames.stream(), Stream.of("d", "e"))
|
||||
.map { name: String -> ColumnId(name, "foo", "bar") }
|
||||
.collect(Collectors.toList()),
|
||||
Optional.empty(),
|
||||
LinkedHashMap(),
|
||||
0,
|
||||
0,
|
||||
0
|
||||
)
|
||||
columnNameMapping =
|
||||
ColumnNameMapping(
|
||||
mapOf(
|
||||
"a1" to "a2",
|
||||
"b1" to "b2",
|
||||
"c1" to "c2",
|
||||
"d1" to "d2",
|
||||
"e1" to "e2",
|
||||
)
|
||||
)
|
||||
Assertions.assertTrue(clusteringMatches(stream, columnNameMapping, existingTable))
|
||||
Assertions.assertTrue(clusteringMatches(stream, existingTable))
|
||||
}
|
||||
|
||||
@Test
|
||||
@@ -225,4 +152,42 @@ class BigqueryDestinationHandlerTest {
|
||||
)
|
||||
Assertions.assertTrue(partitioningMatches(existingTable))
|
||||
}
|
||||
|
||||
@Test
|
||||
fun testSchemaContainAllFinalTableV2AirbyteColumns() {
|
||||
Assertions.assertTrue(
|
||||
schemaContainAllFinalTableV2AirbyteColumns(
|
||||
setOf(
|
||||
"_airbyte_meta",
|
||||
"_airbyte_generation_id",
|
||||
"_airbyte_extracted_at",
|
||||
"_airbyte_raw_id"
|
||||
)
|
||||
)
|
||||
)
|
||||
Assertions.assertFalse(
|
||||
schemaContainAllFinalTableV2AirbyteColumns(
|
||||
setOf("_airbyte_extracted_at", "_airbyte_raw_id")
|
||||
)
|
||||
)
|
||||
Assertions.assertFalse(
|
||||
schemaContainAllFinalTableV2AirbyteColumns(setOf("_airbyte_meta", "_airbyte_raw_id"))
|
||||
)
|
||||
Assertions.assertFalse(
|
||||
schemaContainAllFinalTableV2AirbyteColumns(
|
||||
setOf("_airbyte_meta", "_airbyte_extracted_at")
|
||||
)
|
||||
)
|
||||
Assertions.assertFalse(schemaContainAllFinalTableV2AirbyteColumns(setOf()))
|
||||
Assertions.assertTrue(
|
||||
schemaContainAllFinalTableV2AirbyteColumns(
|
||||
setOf(
|
||||
"_AIRBYTE_META",
|
||||
"_AIRBYTE_GENERATION_ID",
|
||||
"_AIRBYTE_EXTRACTED_AT",
|
||||
"_AIRBYTE_RAW_ID"
|
||||
)
|
||||
)
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
@@ -213,6 +213,7 @@ tutorials:
|
||||
|
||||
| Version | Date | Pull Request | Subject |
|
||||
|:------------|:-----------|:-----------------------------------------------------------|:----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
|
||||
| 2.12.3 | 2025-06-16 | [61648](https://github.com/airbytehq/airbyte/pull/61648) | This is not the release you are looking for. identical to 2.10.2 |
|
||||
| 2.12.2-rc.1 | 2025-06-16 | [61637](https://github.com/airbytehq/airbyte/pull/61637) | 2.12.2 RC 1 (theoretically equivalent to 2.12.0, but with fixed global state handling in CDK) |
|
||||
| 2.12.1 | 2025-06-13 | [61588](https://github.com/airbytehq/airbyte/pull/61588) | ~~Publish version to account for possible duplicate publishing in pipeline. Noop change.~~ WARNING: THIS HAS A BUG. DO NOT USE. |
|
||||
| 2.12.0 | 2025-06-06 | [61432](https://github.com/airbytehq/airbyte/pull/61432) | Improve performance in GCS staging mode by writing GZIP-compressed files. |
|
||||
|
||||
@@ -42,7 +42,6 @@ is_in_whitelist() {
|
||||
local connector="$1"
|
||||
case "$connector" in
|
||||
destination-azure-blob-storage|\
|
||||
destination-bigquery|\
|
||||
destination-csv|\
|
||||
destination-databricks|\
|
||||
destination-dev-null|\
|
||||
|
||||
Reference in New Issue
Block a user